Madan Jampani

MapDB backed Copycat log implementation

...@@ -64,6 +64,12 @@ ...@@ -64,6 +64,12 @@
64 --> 64 -->
65 65
66 <dependency> 66 <dependency>
67 + <groupId>org.mapdb</groupId>
68 + <artifactId>mapdb</artifactId>
69 + <version>1.0.6</version>
70 + </dependency>
71 +
72 + <dependency>
67 <groupId>com.fasterxml.jackson.core</groupId> 73 <groupId>com.fasterxml.jackson.core</groupId>
68 <artifactId>jackson-databind</artifactId> 74 <artifactId>jackson-databind</artifactId>
69 </dependency> 75 </dependency>
......
...@@ -3,12 +3,17 @@ package org.onlab.onos.store.service.impl; ...@@ -3,12 +3,17 @@ package org.onlab.onos.store.service.impl;
3 import static org.slf4j.LoggerFactory.getLogger; 3 import static org.slf4j.LoggerFactory.getLogger;
4 4
5 import java.util.concurrent.CompletableFuture; 5 import java.util.concurrent.CompletableFuture;
6 +import java.util.function.BiConsumer;
6 7
7 import net.kuujo.copycat.protocol.PingRequest; 8 import net.kuujo.copycat.protocol.PingRequest;
9 +import net.kuujo.copycat.protocol.PingResponse;
8 import net.kuujo.copycat.protocol.PollRequest; 10 import net.kuujo.copycat.protocol.PollRequest;
11 +import net.kuujo.copycat.protocol.PollResponse;
9 import net.kuujo.copycat.protocol.RequestHandler; 12 import net.kuujo.copycat.protocol.RequestHandler;
10 import net.kuujo.copycat.protocol.SubmitRequest; 13 import net.kuujo.copycat.protocol.SubmitRequest;
14 +import net.kuujo.copycat.protocol.SubmitResponse;
11 import net.kuujo.copycat.protocol.SyncRequest; 15 import net.kuujo.copycat.protocol.SyncRequest;
16 +import net.kuujo.copycat.protocol.SyncResponse;
12 import net.kuujo.copycat.spi.protocol.ProtocolServer; 17 import net.kuujo.copycat.spi.protocol.ProtocolServer;
13 18
14 import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService; 19 import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
...@@ -57,37 +62,37 @@ public class ClusterMessagingProtocolServer implements ProtocolServer { ...@@ -57,37 +62,37 @@ public class ClusterMessagingProtocolServer implements ProtocolServer {
57 public void handle(ClusterMessage message) { 62 public void handle(ClusterMessage message) {
58 T request = ClusterMessagingProtocol.SERIALIZER.decode(message.payload()); 63 T request = ClusterMessagingProtocol.SERIALIZER.decode(message.payload());
59 if (request.getClass().equals(PingRequest.class)) { 64 if (request.getClass().equals(PingRequest.class)) {
60 - handler.ping((PingRequest) request).whenComplete((response, error) -> { 65 + handler.ping((PingRequest) request).whenComplete(new PostExecutionTask<PingResponse>(message));
61 - try {
62 - message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
63 - } catch (Exception e) {
64 - log.error("Failed to respond to ping request", e);
65 - }
66 - });
67 } else if (request.getClass().equals(PollRequest.class)) { 66 } else if (request.getClass().equals(PollRequest.class)) {
68 - handler.poll((PollRequest) request).whenComplete((response, error) -> { 67 + handler.poll((PollRequest) request).whenComplete(new PostExecutionTask<PollResponse>(message));
69 - try {
70 - message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
71 - } catch (Exception e) {
72 - log.error("Failed to respond to poll request", e);
73 - }
74 - });
75 } else if (request.getClass().equals(SyncRequest.class)) { 68 } else if (request.getClass().equals(SyncRequest.class)) {
76 - handler.sync((SyncRequest) request).whenComplete((response, error) -> { 69 + handler.sync((SyncRequest) request).whenComplete(new PostExecutionTask<SyncResponse>(message));
77 - try {
78 - message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
79 - } catch (Exception e) {
80 - log.error("Failed to respond to sync request", e);
81 - }
82 - });
83 } else if (request.getClass().equals(SubmitRequest.class)) { 70 } else if (request.getClass().equals(SubmitRequest.class)) {
84 - handler.submit((SubmitRequest) request).whenComplete((response, error) -> { 71 + handler.submit((SubmitRequest) request).whenComplete(new PostExecutionTask<SubmitResponse>(message));
72 + } else {
73 + throw new IllegalStateException("Unknown request type: " + request.getClass().getName());
74 + }
75 + }
76 +
77 + private class PostExecutionTask<R> implements BiConsumer<R, Throwable> {
78 +
79 + private final ClusterMessage message;
80 +
81 + public PostExecutionTask(ClusterMessage message) {
82 + this.message = message;
83 + }
84 +
85 + @Override
86 + public void accept(R response, Throwable t) {
87 + if (t != null) {
88 + log.error("Processing for " + message.subject() + " failed.", t);
89 + } else {
85 try { 90 try {
86 message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response)); 91 message.respond(ClusterMessagingProtocol.SERIALIZER.encode(response));
87 } catch (Exception e) { 92 } catch (Exception e) {
88 - log.error("Failed to respond to submit request", e); 93 + log.error("Failed to respond to " + response.getClass().getName(), e);
94 + }
89 } 95 }
90 - });
91 } 96 }
92 } 97 }
93 } 98 }
......
...@@ -172,8 +172,8 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -172,8 +172,8 @@ public class DatabaseStateMachine implements StateMachine {
172 try { 172 try {
173 return SERIALIZER.encode(state); 173 return SERIALIZER.encode(state);
174 } catch (Exception e) { 174 } catch (Exception e) {
175 - log.error("Snapshot serialization error", e); 175 + log.error("Failed to take snapshot", e);
176 - return null; 176 + throw new SnapshotException(e);
177 } 177 }
178 } 178 }
179 179
...@@ -182,7 +182,8 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -182,7 +182,8 @@ public class DatabaseStateMachine implements StateMachine {
182 try { 182 try {
183 this.state = SERIALIZER.decode(data); 183 this.state = SERIALIZER.decode(data);
184 } catch (Exception e) { 184 } catch (Exception e) {
185 - log.error("Snapshot deserialization error", e); 185 + log.error("Failed to install from snapshot", e);
186 + throw new SnapshotException(e);
186 } 187 }
187 } 188 }
188 } 189 }
......
1 +package org.onlab.onos.store.service.impl;
2 +
3 +import static com.google.common.base.Preconditions.checkArgument;
4 +import static com.google.common.base.Preconditions.checkState;
5 +
6 +import java.io.File;
7 +import java.io.IOException;
8 +import java.util.ArrayList;
9 +import java.util.Arrays;
10 +import java.util.List;
11 +import java.util.concurrent.ConcurrentNavigableMap;
12 +
13 +import net.kuujo.copycat.log.Entry;
14 +import net.kuujo.copycat.log.Log;
15 +import net.kuujo.copycat.log.LogIndexOutOfBoundsException;
16 +
17 +import org.mapdb.Atomic;
18 +import org.mapdb.BTreeMap;
19 +import org.mapdb.DB;
20 +import org.mapdb.DBMaker;
21 +import org.mapdb.TxBlock;
22 +import org.mapdb.TxMaker;
23 +import org.onlab.onos.store.serializers.StoreSerializer;
24 +
25 +import com.google.common.collect.Lists;
26 +
27 +/**
28 + * MapDB based log implementation.
29 + */
30 +public class MapDBLog implements Log {
31 +
32 + private final File dbFile;
33 + private TxMaker txMaker;
34 + private final StoreSerializer serializer;
35 + private static final String LOG_NAME = "log";
36 + private static final String SIZE_FIELD_NAME = "size";
37 +
38 + public MapDBLog(File dbFile, StoreSerializer serializer) {
39 + this.dbFile = dbFile;
40 + this.serializer = serializer;
41 + }
42 +
43 + @Override
44 + public void open() throws IOException {
45 + txMaker = DBMaker
46 + .newFileDB(dbFile)
47 + .makeTxMaker();
48 + }
49 +
50 + @Override
51 + public void close() throws IOException {
52 + assertIsOpen();
53 + txMaker.close();
54 + txMaker = null;
55 + }
56 +
57 + @Override
58 + public boolean isOpen() {
59 + return txMaker != null;
60 + }
61 +
62 + protected void assertIsOpen() {
63 + checkState(isOpen(), "The log is not currently open.");
64 + }
65 +
66 + @Override
67 + public long appendEntry(Entry entry) {
68 + checkArgument(entry != null, "expecting non-null entry");
69 + return appendEntries(entry).get(0);
70 + }
71 +
72 + @Override
73 + public List<Long> appendEntries(Entry... entries) {
74 + checkArgument(entries != null, "expecting non-null entries");
75 + return appendEntries(Arrays.asList(entries));
76 + }
77 +
78 + @Override
79 + public List<Long> appendEntries(List<Entry> entries) {
80 + assertIsOpen();
81 + checkArgument(entries != null, "expecting non-null entries");
82 + final List<Long> indices = Lists.newArrayList();
83 +
84 + txMaker.execute(new TxBlock() {
85 + @Override
86 + public void tx(DB db) {
87 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
88 + Atomic.Long size = db.getAtomicLong(SIZE_FIELD_NAME);
89 + long nextIndex = log.isEmpty() ? 1 : log.lastKey() + 1;
90 + for (Entry entry : entries) {
91 + byte[] entryBytes = serializer.encode(entry);
92 + log.put(nextIndex, entryBytes);
93 + size.addAndGet(entryBytes.length);
94 + indices.add(nextIndex);
95 + nextIndex++;
96 + }
97 + }
98 + });
99 +
100 + return indices;
101 + }
102 +
103 + @Override
104 + public boolean containsEntry(long index) {
105 + assertIsOpen();
106 + DB db = txMaker.makeTx();
107 + try {
108 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
109 + return log.containsKey(index);
110 + } finally {
111 + db.close();
112 + }
113 + }
114 +
115 + @Override
116 + public void delete() throws IOException {
117 + assertIsOpen();
118 + txMaker.execute(new TxBlock() {
119 + @Override
120 + public void tx(DB db) {
121 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
122 + Atomic.Long size = db.getAtomicLong(SIZE_FIELD_NAME);
123 + log.clear();
124 + size.set(0);
125 + }
126 + });
127 + }
128 +
129 + @Override
130 + public <T extends Entry> T firstEntry() {
131 + assertIsOpen();
132 + DB db = txMaker.makeTx();
133 + try {
134 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
135 + return log.isEmpty() ? null : serializer.decode(log.firstEntry().getValue());
136 + } finally {
137 + db.close();
138 + }
139 + }
140 +
141 + @Override
142 + public long firstIndex() {
143 + assertIsOpen();
144 + DB db = txMaker.makeTx();
145 + try {
146 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
147 + return log.isEmpty() ? 0 : log.firstKey();
148 + } finally {
149 + db.close();
150 + }
151 + }
152 +
153 + @Override
154 + public <T extends Entry> List<T> getEntries(long from, long to) {
155 + assertIsOpen();
156 + DB db = txMaker.makeTx();
157 + try {
158 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
159 + if (log.isEmpty()) {
160 + throw new LogIndexOutOfBoundsException("Log is empty");
161 + } else if (from < log.firstKey()) {
162 + throw new LogIndexOutOfBoundsException("From index out of bounds.");
163 + } else if (to > log.lastKey()) {
164 + throw new LogIndexOutOfBoundsException("To index out of bounds.");
165 + }
166 + List<T> entries = new ArrayList<>((int) (to - from + 1));
167 + for (long i = from; i <= to; i++) {
168 + T entry = serializer.decode(log.get(i));
169 + entries.add(entry);
170 + }
171 + return entries;
172 + } finally {
173 + db.close();
174 + }
175 + }
176 +
177 + @Override
178 + public <T extends Entry> T getEntry(long index) {
179 + assertIsOpen();
180 + DB db = txMaker.makeTx();
181 + try {
182 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
183 + byte[] entryBytes = log.get(index);
184 + return entryBytes == null ? null : serializer.decode(entryBytes);
185 + } finally {
186 + db.close();
187 + }
188 + }
189 +
190 + @Override
191 + public boolean isEmpty() {
192 + assertIsOpen();
193 + DB db = txMaker.makeTx();
194 + try {
195 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
196 + return log.isEmpty();
197 + } finally {
198 + db.close();
199 + }
200 + }
201 +
202 + @Override
203 + public <T extends Entry> T lastEntry() {
204 + assertIsOpen();
205 + DB db = txMaker.makeTx();
206 + try {
207 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
208 + return log.isEmpty() ? null : serializer.decode(log.lastEntry().getValue());
209 + } finally {
210 + db.close();
211 + }
212 + }
213 +
214 + @Override
215 + public long lastIndex() {
216 + assertIsOpen();
217 + DB db = txMaker.makeTx();
218 + try {
219 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
220 + return log.isEmpty() ? 0 : log.lastKey();
221 + } finally {
222 + db.close();
223 + }
224 + }
225 +
226 + @Override
227 + public void removeAfter(long index) {
228 + assertIsOpen();
229 + txMaker.execute(new TxBlock() {
230 + @Override
231 + public void tx(DB db) {
232 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
233 + Atomic.Long size = db.getAtomicLong(SIZE_FIELD_NAME);
234 + long startIndex = index + 1;
235 + long endIndex = log.lastKey();
236 + for (long i = startIndex; i <= endIndex; ++i) {
237 + byte[] entryBytes = log.remove(i);
238 + size.addAndGet(-1L * entryBytes.length);
239 + }
240 + }
241 + });
242 + }
243 +
244 + @Override
245 + public long size() {
246 + assertIsOpen();
247 + DB db = txMaker.makeTx();
248 + try {
249 + Atomic.Long size = db.getAtomicLong(SIZE_FIELD_NAME);
250 + return size.get();
251 + } finally {
252 + db.close();
253 + }
254 + }
255 +
256 + @Override
257 + public void sync() throws IOException {
258 + assertIsOpen();
259 + }
260 +
261 + @Override
262 + public void compact(long index, Entry entry) throws IOException {
263 +
264 + assertIsOpen();
265 + txMaker.execute(new TxBlock() {
266 + @Override
267 + public void tx(DB db) {
268 + BTreeMap<Long, byte[]> log = db.getTreeMap(LOG_NAME);
269 + Atomic.Long size = db.getAtomicLong(SIZE_FIELD_NAME);
270 + ConcurrentNavigableMap<Long, byte[]> headMap = log.headMap(index);
271 + long deletedBytes = headMap.keySet().stream().mapToLong(i -> log.remove(i).length).sum();
272 + size.addAndGet(-1 * deletedBytes);
273 + byte[] entryBytes = serializer.encode(entry);
274 + byte[] existingEntry = log.put(index, entryBytes);
275 + size.addAndGet(entryBytes.length - existingEntry.length);
276 + db.compact();
277 + }
278 + });
279 + }
280 +}
...\ No newline at end of file ...\ No newline at end of file
1 +package org.onlab.onos.store.service.impl;
2 +
3 +import org.onlab.onos.store.service.DatabaseException;
4 +
5 +/**
6 + * Exception that indicates a problem with the state machine snapshotting.
7 + */
8 +@SuppressWarnings("serial")
9 +public class SnapshotException extends DatabaseException {
10 + public SnapshotException(Throwable t) {
11 + super(t);
12 + }
13 +}
1 +package org.onlab.onos.store.service.impl;
2 +
3 +import java.io.File;
4 +import java.io.IOException;
5 +import java.nio.file.Files;
6 +import java.util.List;
7 +
8 +import net.kuujo.copycat.internal.log.OperationEntry;
9 +import net.kuujo.copycat.log.Entry;
10 +import net.kuujo.copycat.log.Log;
11 +
12 +import org.junit.After;
13 +import org.junit.Assert;
14 +import org.junit.Before;
15 +import org.junit.Test;
16 +import org.onlab.onos.store.serializers.StoreSerializer;
17 +
18 +import com.google.common.testing.EqualsTester;
19 +
20 +/**
21 + * Test the MapDBLog implementation.
22 + */
23 +public class MapDBLogTest {
24 +
25 + private static final String DB_FILE_NAME = "mapdbTest";
26 + private static final StoreSerializer SERIALIZER = ClusterMessagingProtocol.SERIALIZER;
27 + private static final Entry TEST_ENTRY1 = new OperationEntry(1, "test1");
28 + private static final Entry TEST_ENTRY2 = new OperationEntry(2, "test12");
29 + private static final Entry TEST_ENTRY3 = new OperationEntry(3, "test123");
30 + private static final Entry TEST_ENTRY4 = new OperationEntry(4, "test1234");
31 +
32 + private static final Entry TEST_SNAPSHOT_ENTRY = new OperationEntry(5, "snapshot");
33 +
34 + private static final long TEST_ENTRY1_SIZE = SERIALIZER.encode(TEST_ENTRY1).length;
35 + private static final long TEST_ENTRY2_SIZE = SERIALIZER.encode(TEST_ENTRY2).length;
36 + private static final long TEST_ENTRY3_SIZE = SERIALIZER.encode(TEST_ENTRY3).length;
37 + private static final long TEST_ENTRY4_SIZE = SERIALIZER.encode(TEST_ENTRY4).length;
38 +
39 + private static final long TEST_SNAPSHOT_ENTRY_SIZE = SERIALIZER.encode(TEST_SNAPSHOT_ENTRY).length;
40 +
41 + @Before
42 + public void setUp() throws Exception {
43 + }
44 +
45 + @After
46 + public void tearDown() throws Exception {
47 + Files.deleteIfExists(new File(DB_FILE_NAME).toPath());
48 + }
49 +
50 + @Test(expected = IllegalStateException.class)
51 + public void testAssertOpen() {
52 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
53 + log.size();
54 + }
55 +
56 + @Test
57 + public void testAppendEntry() throws IOException {
58 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
59 + log.open();
60 + log.appendEntry(TEST_ENTRY1);
61 + OperationEntry first = log.firstEntry();
62 + OperationEntry last = log.lastEntry();
63 + new EqualsTester()
64 + .addEqualityGroup(first, last, TEST_ENTRY1)
65 + .testEquals();
66 + Assert.assertEquals(TEST_ENTRY1_SIZE, log.size());
67 + Assert.assertEquals(1, log.firstIndex());
68 + Assert.assertEquals(1, log.lastIndex());
69 + }
70 +
71 + @Test
72 + public void testAppendEntries() throws IOException {
73 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
74 + log.open();
75 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3);
76 + OperationEntry first = log.firstEntry();
77 + OperationEntry last = log.lastEntry();
78 + new EqualsTester()
79 + .addEqualityGroup(first, TEST_ENTRY1)
80 + .addEqualityGroup(last, TEST_ENTRY3)
81 + .testEquals();
82 + Assert.assertEquals(TEST_ENTRY1_SIZE + TEST_ENTRY2_SIZE, TEST_ENTRY3_SIZE, log.size());
83 + Assert.assertEquals(1, log.firstIndex());
84 + Assert.assertEquals(3, log.lastIndex());
85 + Assert.assertTrue(log.containsEntry(1));
86 + Assert.assertTrue(log.containsEntry(2));
87 + }
88 +
89 + @Test
90 + public void testDelete() throws IOException {
91 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
92 + log.open();
93 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2);
94 + log.delete();
95 + Assert.assertEquals(0, log.size());
96 + Assert.assertTrue(log.isEmpty());
97 + Assert.assertEquals(0, log.firstIndex());
98 + Assert.assertNull(log.firstEntry());
99 + Assert.assertEquals(0, log.lastIndex());
100 + Assert.assertNull(log.lastEntry());
101 + }
102 +
103 + @Test
104 + public void testGetEntries() throws IOException {
105 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
106 + log.open();
107 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3, TEST_ENTRY4);
108 + Assert.assertEquals(
109 + TEST_ENTRY1_SIZE +
110 + TEST_ENTRY2_SIZE +
111 + TEST_ENTRY3_SIZE +
112 + TEST_ENTRY4_SIZE, log.size());
113 +
114 + List<Entry> entries = log.getEntries(2, 3);
115 + new EqualsTester()
116 + .addEqualityGroup(log.getEntry(4), TEST_ENTRY4)
117 + .addEqualityGroup(entries.get(0), TEST_ENTRY2)
118 + .addEqualityGroup(entries.get(1), TEST_ENTRY3)
119 + .testEquals();
120 + }
121 +
122 + @Test
123 + public void testRemoveAfter() throws IOException {
124 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
125 + log.open();
126 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3, TEST_ENTRY4);
127 + log.removeAfter(1);
128 + Assert.assertEquals(TEST_ENTRY1_SIZE, log.size());
129 + new EqualsTester()
130 + .addEqualityGroup(log.firstEntry(), log.lastEntry(), TEST_ENTRY1)
131 + .testEquals();
132 + }
133 +
134 + @Test
135 + public void testAddAfterRemove() throws IOException {
136 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
137 + log.open();
138 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3, TEST_ENTRY4);
139 + log.removeAfter(1);
140 + log.appendEntry(TEST_ENTRY4);
141 + Assert.assertEquals(TEST_ENTRY1_SIZE + TEST_ENTRY4_SIZE, log.size());
142 + new EqualsTester()
143 + .addEqualityGroup(log.firstEntry(), TEST_ENTRY1)
144 + .addEqualityGroup(log.lastEntry(), TEST_ENTRY4)
145 + .addEqualityGroup(log.size(), TEST_ENTRY1_SIZE + TEST_ENTRY4_SIZE)
146 + .testEquals();
147 + }
148 +
149 + @Test
150 + public void testClose() throws IOException {
151 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
152 + Assert.assertFalse(log.isOpen());
153 + log.open();
154 + Assert.assertTrue(log.isOpen());
155 + log.close();
156 + Assert.assertFalse(log.isOpen());
157 + }
158 +
159 + @Test
160 + public void testReopen() throws IOException {
161 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
162 + log.open();
163 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3, TEST_ENTRY4);
164 + log.close();
165 + log.open();
166 +
167 + new EqualsTester()
168 + .addEqualityGroup(log.firstEntry(), TEST_ENTRY1)
169 + .addEqualityGroup(log.getEntry(2), TEST_ENTRY2)
170 + .addEqualityGroup(log.lastEntry(), TEST_ENTRY4)
171 + .addEqualityGroup(log.size(),
172 + TEST_ENTRY1_SIZE +
173 + TEST_ENTRY2_SIZE +
174 + TEST_ENTRY3_SIZE +
175 + TEST_ENTRY4_SIZE)
176 + .testEquals();
177 + }
178 +
179 + @Test
180 + public void testCompact() throws IOException {
181 + Log log = new MapDBLog(new File(DB_FILE_NAME), SERIALIZER);
182 + log.open();
183 + log.appendEntries(TEST_ENTRY1, TEST_ENTRY2, TEST_ENTRY3, TEST_ENTRY4);
184 + log.compact(3, TEST_SNAPSHOT_ENTRY);
185 + new EqualsTester()
186 + .addEqualityGroup(log.firstEntry(), TEST_SNAPSHOT_ENTRY)
187 + .addEqualityGroup(log.lastEntry(), TEST_ENTRY4)
188 + .addEqualityGroup(log.size(),
189 + TEST_SNAPSHOT_ENTRY_SIZE +
190 + TEST_ENTRY4_SIZE)
191 + .testEquals();
192 + }
193 +}