From 93d6ff11432361a9be33392aac2312e8ec03ff27 Mon Sep 17 00:00:00 2001 From: TheR1sing3un <87409330+TheR1sing3un@users.noreply.github.com> Date: Sat, 15 Jul 2023 21:12:30 +0800 Subject: [PATCH] [ISSUE #275] Complete snapshot(sync mode to install) (#298) * feat(core): support protocol about install snapshot 1. support protocol about install snapshot Closes https://github.com/openmessaging/dledger/issues/275 * feat(core): support protocol about install snapshot 1. support protocol about install snapshot Closes https://github.com/openmessaging/dledger/issues/275 * feat(core): refactor basic dledger overall structure to make it more "raft" like 1. refactor basic dledger overall structure to make it more "raft" like * feat(core): pass all original test 1. pass all original test * feat(core): support batch append 1. support batch append * fix(example): resolve conflicts after rebasing master 1. resolve conflicts after rebasing master * fix(jepsen): resolve conflicts about jepsen after rebasing master 1. resolve conflicts about jepsen after rebasing master * fix(jepsen): fix type error 1. fix type error * feat(core): support installing snapshot 1. support installing snapshot * feat(core): support installing snapshot 1. support installing snapshot * feat(jepsen): test snapshot in jepsen 1. test snapshot in jepsen * test(core): polish flaky test 1. polish flaky test * rerun * feat(core): commit entry which is in current term 1. commit entry which is in current term * rerun * rerun * rerun * fix(core): make the field: position in DLedgerEntry meaningless 1. make the field: position in DLedgerEntry meaningless * test(core): use different store base path for each ut 1. use different store base path for each ut * test(core): use different store base path for each ut 1. use different store base path for each ut * rerun * test(core): use different store base path for each ut 1. use different store base path for each ut * test(core): use different store base path for each ut 1. use different store base path for each ut * test(core): use different store base path for each ut 1. use different store base path for each ut * fix(core): update peer watermark when compare success 1. update peer watermark when compare success * fix(core): fix 1. fix * fix(core): fix 1. fix * test(proxy): remove proxy test 1. remove proxy test * feat(example): add batch append cmd 1. add batch append cmd * fix(core): reuse forks 1. reuse forks * chore(global): add more git ignore file 1. add more git ignore file * build(global): set reuseForks to false 1. set reuseForks to false * rerun * feat(core): clear pending map and set writeIndex when role dispatcher role change from append to compare 1. clear pending map and set writeIndex when role dispatcher role change from append to compare * rerun --- .gitignore | 4 +- .../storage/dledger/DLedgerConfig.java | 69 +- .../storage/dledger/DLedgerEntryPusher.java | 921 ++++++++++-------- .../storage/dledger/DLedgerLeaderElector.java | 4 +- .../dledger/DLedgerRpcNettyService.java | 50 +- .../storage/dledger/DLedgerServer.java | 68 +- .../storage/dledger/MemberState.java | 79 +- .../storage/dledger/client/DLedgerClient.java | 20 +- .../dledger/{ => common}/AppendFuture.java | 4 +- .../{ => common}/BatchAppendFuture.java | 4 +- .../storage/dledger/{ => common}/Closure.java | 6 +- .../{ => common}/NamedThreadFactory.java | 4 +- .../dledger/{ => common}/ReadClosure.java | 4 +- .../dledger/{ => common}/ReadMode.java | 4 +- .../{ => common}/ShutdownAbleThread.java | 4 +- .../storage/dledger/{ => common}/Status.java | 4 +- .../dledger/{ => common}/TimeoutFuture.java | 4 +- .../dledger/entry/DLedgerEntryCoder.java | 10 + .../dledger/entry/DLedgerIndexEntry.java | 70 ++ .../dledger/protocol/DLedgerRequestCode.java | 3 + .../dledger/protocol/DLedgerResponseCode.java | 4 +- .../protocol/InstallSnapshotRequest.java | 67 ++ .../protocol/InstallSnapshotResponse.java | 29 + .../dledger/protocol/PushEntryRequest.java | 76 +- .../dledger/protocol/PushEntryResponse.java | 20 + .../handler/DLedgerInnerProtocolHandler.java | 4 +- .../handler/DLedgerRaftProtocolHandler.java | 4 + .../protocol/DLedgerRaftProtocol.java | 4 + .../dledger/snapshot/DownloadSnapshot.java | 48 + .../snapshot/SnapshotEntryResetStrategy.java | 50 + .../dledger/snapshot/SnapshotManager.java | 198 +++- .../dledger/snapshot/SnapshotReader.java | 2 + .../dledger/snapshot/SnapshotStore.java | 2 + .../snapshot/file/FileSnapshotReader.java | 30 + .../snapshot/file/FileSnapshotStore.java | 47 +- .../snapshot/hook/SaveSnapshotHook.java | 3 - .../statemachine/CommittedEntryIterator.java | 7 +- .../statemachine/NoOpStatemachine.java | 55 ++ .../statemachine/StateMachineCaller.java | 67 +- .../dledger/store/DLedgerMemoryStore.java | 62 +- .../storage/dledger/store/DLedgerStore.java | 35 +- .../store/file/DLedgerMmapFileStore.java | 280 +++--- .../dledger/store/file/MmapFileList.java | 2 +- .../storage/dledger/AppendAndGetTest.java | 9 + .../storage/dledger/AppendAndPushTest.java | 18 +- .../storage/dledger/AppendAndReadTest.java | 62 +- .../storage/dledger/BatchPushTest.java | 11 +- .../storage/dledger/LeaderElectorTest.java | 10 +- .../storage/dledger/ServerTestHarness.java | 22 +- .../{ => common}/NamedThreadFactoryTest.java | 5 +- .../protocol/HeartbeatRequestTest.java | 7 + .../dledger/protocol/VoteRequestTest.java | 8 + .../dledger/snapshot/SnapshotManagerTest.java | 167 +++- .../dledger/snapshot/SnapshotStoreTest.java | 8 +- .../statemachine/MockStateMachine.java | 20 +- .../statemachine/StateMachineCallerTest.java | 16 +- .../register/RegisterReadProcessor.java | 7 +- .../register/RegisterStateMachine.java | 6 +- .../DLedgerManagedFileStoreTruncateTest.java | 9 +- .../store/DLedgerMappedFileStoreTest.java | 137 ++- .../register/RegisterStateMachine.java | 6 +- .../client/RegisterDLedgerClient.java | 16 +- .../example/register/command/ReadCommand.java | 4 +- .../register/command/RegisterCommand.java | 22 +- .../protocol/RegisterReadProcessor.java | 4 +- .../protocol/RegisterReadRequest.java | 16 +- .../protocol/RegisterWriteRequest.java | 14 +- jepsen/build.sh | 2 +- .../storage/dledger/jepsen/core.clj | 26 +- pom.xml | 2 +- .../storage/dledger/proxy/DLedgerProxy.java | 15 +- .../dledger/proxy/ProxyAppendAndGetTest.java | 404 -------- .../dledger/proxy/ProxyLeaderElectorTest.java | 309 ------ .../storage/dledger/proxy/ServerTestBase.java | 46 - .../dledger/proxy/ServerTestHarness.java | 229 ----- 75 files changed, 2185 insertions(+), 1884 deletions(-) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/AppendFuture.java (91%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/BatchAppendFuture.java (92%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/Closure.java (85%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/NamedThreadFactory.java (95%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/ReadClosure.java (86%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/ReadMode.java (86%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/ShutdownAbleThread.java (96%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/Status.java (91%) rename dledger/src/main/java/io/openmessaging/storage/dledger/{ => common}/TimeoutFuture.java (92%) create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerIndexEntry.java create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotRequest.java create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotResponse.java create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/DownloadSnapshot.java create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotEntryResetStrategy.java create mode 100644 dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/NoOpStatemachine.java rename dledger/src/test/java/io/openmessaging/storage/dledger/{ => common}/NamedThreadFactoryTest.java (91%) delete mode 100644 proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyAppendAndGetTest.java delete mode 100644 proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyLeaderElectorTest.java delete mode 100644 proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestBase.java delete mode 100644 proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestHarness.java diff --git a/.gitignore b/.gitignore index 9e08b3f6..b90eed6d 100644 --- a/.gitignore +++ b/.gitignore @@ -14,4 +14,6 @@ devenv .DS_Store nohup.out *.gz -dledger-example.jar \ No newline at end of file +dledger-example.jar +.clj-kondo +*.lsp diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerConfig.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerConfig.java index dbc72cb4..78020ab3 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerConfig.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerConfig.java @@ -16,6 +16,7 @@ package io.openmessaging.storage.dledger; +import io.openmessaging.storage.dledger.snapshot.SnapshotEntryResetStrategy; import io.openmessaging.storage.dledger.store.file.DLedgerMmapFileStore; import io.openmessaging.storage.dledger.utils.DLedgerUtils; @@ -86,16 +87,31 @@ public class DLedgerConfig { private int minTakeLeadershipVoteIntervalMs = 30; private int maxTakeLeadershipVoteIntervalMs = 100; - private boolean isEnableBatchPush = false; - private int maxBatchPushSize = 4 * 1024; + private boolean isEnableBatchAppend = false; + + // max size in bytes for each append request + private int maxBatchAppendSize = 4 * 1024; private long leadershipTransferWaitTimeout = 1000; private boolean enableSnapshot = false; + private SnapshotEntryResetStrategy snapshotEntryResetStrategy = SnapshotEntryResetStrategy.RESET_ALL_SYNC; + private int snapshotThreshold = 1000; + + private int resetSnapshotEntriesDelayTime = 5 * 1000; + + /** + * reset snapshot entries but keep last entries num. + * .e.g 10, when we load from snapshot which lastIncludedIndex = 100, we will delete the entries in (..., 90] + */ + private int resetSnapshotEntriesButKeepLastEntriesNum = 10; private int maxSnapshotReservedNum = 3; + // max interval in ms for each append request + private int maxBatchAppendIntervalMs = 1000; + public String getDefaultPath() { return storeBaseDir + File.separator + "dledger-" + selfId; } @@ -394,20 +410,20 @@ public void setMaxTakeLeadershipVoteIntervalMs(int maxTakeLeadershipVoteInterval this.maxTakeLeadershipVoteIntervalMs = maxTakeLeadershipVoteIntervalMs; } - public boolean isEnableBatchPush() { - return isEnableBatchPush; + public boolean isEnableBatchAppend() { + return isEnableBatchAppend; } - public void setEnableBatchPush(boolean enableBatchPush) { - isEnableBatchPush = enableBatchPush; + public void setEnableBatchAppend(boolean enableBatchAppend) { + isEnableBatchAppend = enableBatchAppend; } - public int getMaxBatchPushSize() { - return maxBatchPushSize; + public int getMaxBatchAppendSize() { + return maxBatchAppendSize; } - public void setMaxBatchPushSize(int maxBatchPushSize) { - this.maxBatchPushSize = maxBatchPushSize; + public void setMaxBatchAppendSize(int maxBatchAppendSize) { + this.maxBatchAppendSize = maxBatchAppendSize; } public long getLeadershipTransferWaitTimeout() { @@ -495,4 +511,37 @@ public boolean isEnableSnapshot() { public void setEnableSnapshot(boolean enableSnapshot) { this.enableSnapshot = enableSnapshot; } + + public int getMaxBatchAppendIntervalMs() { + return maxBatchAppendIntervalMs; + } + + public SnapshotEntryResetStrategy getSnapshotEntryResetStrategy() { + return snapshotEntryResetStrategy; + } + + public void setSnapshotEntryResetStrategy( + SnapshotEntryResetStrategy snapshotEntryResetStrategy) { + this.snapshotEntryResetStrategy = snapshotEntryResetStrategy; + } + + public void setMaxBatchAppendIntervalMs(int maxBatchAppendIntervalMs) { + this.maxBatchAppendIntervalMs = maxBatchAppendIntervalMs; + } + + public int getResetSnapshotEntriesDelayTime() { + return resetSnapshotEntriesDelayTime; + } + + public void setResetSnapshotEntriesDelayTime(int resetSnapshotEntriesDelayTime) { + this.resetSnapshotEntriesDelayTime = resetSnapshotEntriesDelayTime; + } + + public int getResetSnapshotEntriesButKeepLastEntriesNum() { + return resetSnapshotEntriesButKeepLastEntriesNum; + } + + public void setResetSnapshotEntriesButKeepLastEntriesNum(int resetSnapshotEntriesButKeepLastEntriesNum) { + this.resetSnapshotEntriesButKeepLastEntriesNum = resetSnapshotEntriesButKeepLastEntriesNum; + } } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerEntryPusher.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerEntryPusher.java index 3777554c..0d12492c 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerEntryPusher.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerEntryPusher.java @@ -17,15 +17,23 @@ package io.openmessaging.storage.dledger; import com.alibaba.fastjson.JSON; +import io.openmessaging.storage.dledger.common.Closure; +import io.openmessaging.storage.dledger.common.ShutdownAbleThread; +import io.openmessaging.storage.dledger.common.Status; +import io.openmessaging.storage.dledger.common.TimeoutFuture; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.exception.DLedgerException; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.PushEntryRequest; import io.openmessaging.storage.dledger.protocol.PushEntryResponse; +import io.openmessaging.storage.dledger.snapshot.DownloadSnapshot; +import io.openmessaging.storage.dledger.snapshot.SnapshotManager; +import io.openmessaging.storage.dledger.snapshot.SnapshotMeta; +import io.openmessaging.storage.dledger.snapshot.SnapshotReader; import io.openmessaging.storage.dledger.statemachine.StateMachineCaller; -import io.openmessaging.storage.dledger.store.DLedgerMemoryStore; import io.openmessaging.storage.dledger.store.DLedgerStore; -import io.openmessaging.storage.dledger.store.file.DLedgerMmapFileStore; import io.openmessaging.storage.dledger.utils.DLedgerUtils; import io.openmessaging.storage.dledger.utils.Pair; import io.openmessaging.storage.dledger.utils.PreConditions; @@ -36,7 +44,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; @@ -44,6 +51,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -60,21 +68,24 @@ public class DLedgerEntryPusher { private final DLedgerRpcService dLedgerRpcService; - private final Map> peerWaterMarksByTerm = new ConcurrentHashMap<>(); + private final Map> peerWaterMarksByTerm = new ConcurrentHashMap<>(); - private final Map> pendingClosure = new ConcurrentHashMap<>(); + private final Map> pendingClosure = new ConcurrentHashMap<>(); private final EntryHandler entryHandler; private final QuorumAckChecker quorumAckChecker; - private final Map dispatcherMap = new HashMap<>(); + private final Map dispatcherMap = new HashMap<>(); - private Optional fsmCaller; + private final String selfId; + + private StateMachineCaller fsmCaller; public DLedgerEntryPusher(DLedgerConfig dLedgerConfig, MemberState memberState, DLedgerStore dLedgerStore, - DLedgerRpcService dLedgerRpcService) { + DLedgerRpcService dLedgerRpcService) { this.dLedgerConfig = dLedgerConfig; + this.selfId = this.dLedgerConfig.getSelfId(); this.memberState = memberState; this.dLedgerStore = dLedgerStore; this.dLedgerRpcService = dLedgerRpcService; @@ -85,7 +96,6 @@ public DLedgerEntryPusher(DLedgerConfig dLedgerConfig, MemberState memberState, } this.entryHandler = new EntryHandler(LOGGER); this.quorumAckChecker = new QuorumAckChecker(LOGGER); - this.fsmCaller = Optional.empty(); } public void startup() { @@ -104,7 +114,7 @@ public void shutdown() { } } - public void registerStateMachine(final Optional fsmCaller) { + public void registerStateMachine(final StateMachineCaller fsmCaller) { this.fsmCaller = fsmCaller; } @@ -112,6 +122,10 @@ public CompletableFuture handlePush(PushEntryRequest request) return entryHandler.handlePush(request); } + public CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) { + return entryHandler.handleInstallSnapshot(request); + } + private void checkTermForWaterMark(long term, String env) { if (!peerWaterMarksByTerm.containsKey(term)) { LOGGER.info("Initialize the watermark in {} for term={}", env, term); @@ -221,11 +235,6 @@ private void checkResponseFuturesElapsed(final long endIndex) { } } - private void updateCommittedIndex(final long term, final long committedIndex) { - dLedgerStore.updateCommittedIndex(term, committedIndex); - this.fsmCaller.ifPresent(caller -> caller.onCommitted(dLedgerStore.getCommittedIndex())); - } - /** * This thread will check the quorum index and complete the pending requests. */ @@ -233,7 +242,6 @@ private class QuorumAckChecker extends ShutdownAbleThread { private long lastPrintWatermarkTimeMs = System.currentTimeMillis(); private long lastCheckLeakTimeMs = System.currentTimeMillis(); - private long lastQuorumIndex = -1; public QuorumAckChecker(Logger logger) { super("QuorumAckChecker-" + memberState.getSelfId(), logger); @@ -243,14 +251,8 @@ public QuorumAckChecker(Logger logger) { public void doWork() { try { if (DLedgerUtils.elapsed(lastPrintWatermarkTimeMs) > 3000) { - if (DLedgerEntryPusher.this.fsmCaller.isPresent()) { - final long lastAppliedIndex = DLedgerEntryPusher.this.fsmCaller.get().getLastAppliedIndex(); - logger.info("[{}][{}] term={} ledgerBeforeBegin={} ledgerEnd={} committed={} watermarks={} appliedIndex={}", - memberState.getSelfId(), memberState.getRole(), memberState.currTerm(), dLedgerStore.getLedgerBeforeBeginIndex(), dLedgerStore.getLedgerEndIndex(), dLedgerStore.getCommittedIndex(), JSON.toJSONString(peerWaterMarksByTerm), lastAppliedIndex); - } else { - logger.info("[{}][{}] term={} ledgerBeforeBegin={} ledgerEnd={} committed={} watermarks={}", - memberState.getSelfId(), memberState.getRole(), memberState.currTerm(), dLedgerStore.getLedgerBeforeBeginIndex(), dLedgerStore.getLedgerEndIndex(), dLedgerStore.getCommittedIndex(), JSON.toJSONString(peerWaterMarksByTerm)); - } + logger.info("[{}][{}] term={} ledgerBeforeBegin={} ledgerEnd={} committed={} watermarks={} appliedIndex={}", + memberState.getSelfId(), memberState.getRole(), memberState.currTerm(), dLedgerStore.getLedgerBeforeBeginIndex(), dLedgerStore.getLedgerEndIndex(), memberState.getCommittedIndex(), JSON.toJSONString(peerWaterMarksByTerm), memberState.getAppliedIndex()); lastPrintWatermarkTimeMs = System.currentTimeMillis(); } if (!memberState.isLeader()) { @@ -260,6 +262,7 @@ public void doWork() { long currTerm = memberState.currTerm(); checkTermForPendingMap(currTerm, "QuorumAckChecker"); checkTermForWaterMark(currTerm, "QuorumAckChecker"); + // clear pending closure in old term if (pendingClosure.size() > 1) { for (Long term : pendingClosure.keySet()) { if (term == currTerm) { @@ -272,6 +275,7 @@ public void doWork() { pendingClosure.remove(term); } } + // clear peer watermarks in old term if (peerWaterMarksByTerm.size() > 1) { for (Long term : peerWaterMarksByTerm.keySet()) { if (term == currTerm) { @@ -282,67 +286,35 @@ public void doWork() { } } + // clear the pending closure which index <= applyIndex + if (DLedgerUtils.elapsed(lastCheckLeakTimeMs) > 1000) { + checkResponseFuturesElapsed(DLedgerEntryPusher.this.memberState.getAppliedIndex()); + lastCheckLeakTimeMs = System.currentTimeMillis(); + } + + // clear the timeout pending closure which index > appliedIndex + checkResponseFuturesTimeout(DLedgerEntryPusher.this.memberState.getAppliedIndex() + 1); + + // update peer watermarks of self + updatePeerWaterMark(currTerm, memberState.getSelfId(), dLedgerStore.getLedgerEndIndex()); + + // calculate the median of watermarks(which we can ensure that more than half of the nodes have been pushed the corresponding entry) + // we can also call it quorumIndex Map peerWaterMarks = peerWaterMarksByTerm.get(currTerm); List sortedWaterMarks = peerWaterMarks.values() - .stream() - .sorted(Comparator.reverseOrder()) - .collect(Collectors.toList()); + .stream() + .sorted(Comparator.reverseOrder()) + .collect(Collectors.toList()); long quorumIndex = sortedWaterMarks.get(sortedWaterMarks.size() / 2); - final Optional fsmCaller = DLedgerEntryPusher.this.fsmCaller; - if (fsmCaller.isPresent()) { - // If there exist statemachine - final StateMachineCaller caller = fsmCaller.get(); - if (quorumIndex > this.lastQuorumIndex) { - DLedgerEntryPusher.this.dLedgerStore.updateCommittedIndex(currTerm, quorumIndex); - caller.onCommitted(quorumIndex); - } - // Check elapsed - if (DLedgerUtils.elapsed(lastCheckLeakTimeMs) > 1000) { - updatePeerWaterMark(currTerm, memberState.getSelfId(), dLedgerStore.getLedgerEndIndex()); - checkResponseFuturesElapsed(caller.getLastAppliedIndex()); - lastCheckLeakTimeMs = System.currentTimeMillis(); - } - if (quorumIndex == this.lastQuorumIndex) { - waitForRunning(1); - } + // advance the commit index + // we can only commit the index whose term is equals to current term (refer to raft paper 5.4.2) + if (DLedgerEntryPusher.this.memberState.leaderUpdateCommittedIndex(currTerm, quorumIndex)) { + DLedgerEntryPusher.this.fsmCaller.onCommitted(quorumIndex); } else { - if (quorumIndex > this.lastQuorumIndex) { - dLedgerStore.updateCommittedIndex(currTerm, quorumIndex); - } - ConcurrentMap closureMap = pendingClosure.get(currTerm); - boolean needCheck = false; - int ackNum = 0; - for (long i = lastQuorumIndex + 1; i <= quorumIndex; i++) { - try { - Closure closure = closureMap.remove(i); - if (closure == null) { - needCheck = true; - continue; - } - if (closure.isTimeOut()) { - closure.done(Status.error(DLedgerResponseCode.TIMEOUT)); - } else { - closure.done(Status.ok()); - ackNum++; - } - } catch (Throwable t) { - logger.error("Error in ack to index={} term={}", i, currTerm, t); - } - } - - if (ackNum == 0) { - checkResponseFuturesTimeout(quorumIndex + 1); - waitForRunning(1); - } - - if (DLedgerUtils.elapsed(lastCheckLeakTimeMs) > 1000 || needCheck) { - updatePeerWaterMark(currTerm, memberState.getSelfId(), dLedgerStore.getLedgerEndIndex()); - checkResponseFuturesElapsed(quorumIndex); - lastCheckLeakTimeMs = System.currentTimeMillis(); - } + // If the commit index is not advanced, we should wait for the next round + waitForRunning(1); } - lastQuorumIndex = quorumIndex; } catch (Throwable t) { DLedgerEntryPusher.LOGGER.error("Error in {}", getName(), t); DLedgerUtils.sleep(100); @@ -369,18 +341,30 @@ public void doWork() { */ private class EntryDispatcher extends ShutdownAbleThread { - private final AtomicReference type = new AtomicReference<>(PushEntryRequest.Type.COMPARE); + private final AtomicReference type = new AtomicReference<>(EntryDispatcherState.COMPARE); private long lastPushCommitTimeMs = -1; private final String peerId; - private long compareIndex = -1; - private long writeIndex = -1; + + /** + * the index of the next entry to push(initialized to the next of the last entry in the store) + */ + private long writeIndex = DLedgerEntryPusher.this.dLedgerStore.getLedgerEndIndex() + 1; + + /** + * the index of the last entry to be pushed to this peer(initialized to -1) + */ + private long matchIndex = -1; + private final int maxPendingSize = 1000; private long term = -1; private String leaderId = null; private long lastCheckLeakTimeMs = System.currentTimeMillis(); - private final ConcurrentMap pendingMap = new ConcurrentHashMap<>(); - private final ConcurrentMap> batchPendingMap = new ConcurrentHashMap<>(); + + private final ConcurrentMap> pendingMap = new ConcurrentHashMap<>(); private final PushEntryRequest batchAppendEntryRequest = new PushEntryRequest(); + + private long lastAppendEntryRequestSendTimeMs = -1; + private final Quota quota = new Quota(dLedgerConfig.getPeerPushQuota()); public EntryDispatcher(String peerId, Logger logger) { @@ -388,6 +372,13 @@ public EntryDispatcher(String peerId, Logger logger) { this.peerId = peerId; } + @Override + public synchronized void start() { + super.start(); + // initialize write index + writeIndex = DLedgerEntryPusher.this.dLedgerStore.getLedgerEndIndex() + 1; + } + private boolean checkNotLeaderAndFreshState() { if (!memberState.isLeader()) { return true; @@ -398,24 +389,52 @@ private boolean checkNotLeaderAndFreshState() { return true; } PreConditions.check(memberState.getSelfId().equals(memberState.getLeaderId()), DLedgerResponseCode.UNKNOWN); + logger.info("[Push-{}->{}]Update term: {} and leaderId: {} to new term: {}, new leaderId: {}", selfId, peerId, term, leaderId, memberState.currTerm(), memberState.getLeaderId()); term = memberState.currTerm(); leaderId = memberState.getSelfId(); - changeState(-1, PushEntryRequest.Type.COMPARE); + changeState(EntryDispatcherState.COMPARE); } } return false; } - private PushEntryRequest buildPushRequest(DLedgerEntry entry, PushEntryRequest.Type target) { + private PushEntryRequest buildCompareOrTruncatePushRequest(long preLogTerm, long preLogIndex, + PushEntryRequest.Type type) { PushEntryRequest request = new PushEntryRequest(); request.setGroup(memberState.getGroup()); request.setRemoteId(peerId); request.setLeaderId(leaderId); request.setLocalId(memberState.getSelfId()); request.setTerm(term); - request.setEntry(entry); - request.setType(target); - request.setCommitIndex(dLedgerStore.getCommittedIndex()); + request.setPreLogIndex(preLogIndex); + request.setPreLogTerm(preLogTerm); + request.setType(type); + request.setCommitIndex(memberState.getCommittedIndex()); + return request; + } + + private PushEntryRequest buildCommitPushRequest() { + PushEntryRequest request = new PushEntryRequest(); + request.setGroup(memberState.getGroup()); + request.setRemoteId(peerId); + request.setLeaderId(leaderId); + request.setLocalId(memberState.getSelfId()); + request.setTerm(term); + request.setType(PushEntryRequest.Type.COMMIT); + request.setCommitIndex(memberState.getCommittedIndex()); + return request; + } + + private InstallSnapshotRequest buildInstallSnapshotRequest(DownloadSnapshot snapshot) { + InstallSnapshotRequest request = new InstallSnapshotRequest(); + request.setGroup(memberState.getGroup()); + request.setRemoteId(peerId); + request.setLeaderId(leaderId); + request.setLocalId(memberState.getSelfId()); + request.setTerm(term); + request.setLastIncludedIndex(snapshot.getMeta().getLastIncludedIndex()); + request.setLastIncludedTerm(snapshot.getMeta().getLastIncludedTerm()); + request.setData(snapshot.getData()); return request; } @@ -433,13 +452,6 @@ private void checkQuotaAndWait(DLedgerEntry entry) { if (dLedgerStore.getLedgerEndIndex() - entry.getIndex() <= maxPendingSize) { return; } - if (dLedgerStore instanceof DLedgerMemoryStore) { - return; - } - DLedgerMmapFileStore mmapFileStore = (DLedgerMmapFileStore) dLedgerStore; - if (mmapFileStore.getDataFileList().getMaxWrotePosition() - entry.getPos() < dLedgerConfig.getPeerPushThrottlePoint()) { - return; - } quota.sample(entry.getSize()); if (quota.validateNow()) { long leftNow = quota.leftNow(); @@ -448,40 +460,6 @@ private void checkQuotaAndWait(DLedgerEntry entry) { } } - private void doAppendInner(long index) throws Exception { - DLedgerEntry entry = getDLedgerEntryForAppend(index); - if (null == entry) { - return; - } - checkQuotaAndWait(entry); - PushEntryRequest request = buildPushRequest(entry, PushEntryRequest.Type.APPEND); - CompletableFuture responseFuture = dLedgerRpcService.push(request); - pendingMap.put(index, System.currentTimeMillis()); - responseFuture.whenComplete((x, ex) -> { - try { - PreConditions.check(ex == null, DLedgerResponseCode.UNKNOWN); - DLedgerResponseCode responseCode = DLedgerResponseCode.valueOf(x.getCode()); - switch (responseCode) { - case SUCCESS: - pendingMap.remove(x.getIndex()); - updatePeerWaterMark(x.getTerm(), peerId, x.getIndex()); - quorumAckChecker.wakeup(); - break; - case INCONSISTENT_STATE: - logger.info("[Push-{}]Get INCONSISTENT_STATE when push index={} term={}", peerId, x.getIndex(), x.getTerm()); - changeState(-1, PushEntryRequest.Type.COMPARE); - break; - default: - logger.warn("[Push-{}]Get error response code {} {}", peerId, responseCode, x.baseInfo()); - break; - } - } catch (Throwable t) { - logger.error("", t); - } - }); - lastPushCommitTimeMs = System.currentTimeMillis(); - } - private DLedgerEntry getDLedgerEntryForAppend(long index) { DLedgerEntry entry; try { @@ -490,7 +468,6 @@ private DLedgerEntry getDLedgerEntryForAppend(long index) { // Do compare, in case the ledgerBeginIndex get refreshed. if (DLedgerResponseCode.INDEX_LESS_THAN_LOCAL_BEGIN.equals(e.getCode())) { logger.info("[Push-{}]Get INDEX_LESS_THAN_LOCAL_BEGIN when requested index is {}, try to compare", peerId, index); - changeState(-1, PushEntryRequest.Type.COMPARE); return null; } throw e; @@ -501,7 +478,7 @@ private DLedgerEntry getDLedgerEntryForAppend(long index) { private void doCommit() throws Exception { if (DLedgerUtils.elapsed(lastPushCommitTimeMs) > 1000) { - PushEntryRequest request = buildPushRequest(null, PushEntryRequest.Type.COMMIT); + PushEntryRequest request = buildCommitPushRequest(); //Ignore the results dLedgerRpcService.push(request); lastPushCommitTimeMs = System.currentTimeMillis(); @@ -510,31 +487,173 @@ private void doCommit() throws Exception { private void doCheckAppendResponse() throws Exception { long peerWaterMark = getPeerWaterMark(term, peerId); - Long sendTimeMs = pendingMap.get(peerWaterMark + 1); - if (sendTimeMs != null && System.currentTimeMillis() - sendTimeMs > dLedgerConfig.getMaxPushTimeOutMs()) { - logger.warn("[Push-{}]Retry to push entry at {}", peerId, peerWaterMark + 1); - doAppendInner(peerWaterMark + 1); + Pair pair = pendingMap.get(peerWaterMark + 1); + if (pair == null) + return; + long sendTimeMs = pair.getKey(); + if (DLedgerUtils.elapsed(sendTimeMs) > dLedgerConfig.getMaxPushTimeOutMs()) { + // reset write index + batchAppendEntryRequest.clear(); + writeIndex = peerWaterMark + 1; + logger.warn("[Push-{}]Reset write index to {} for resending the entries which are timeout", peerId, peerWaterMark + 1); + } + } + + private synchronized void changeState(EntryDispatcherState target) { + logger.info("[Push-{}]Change state from {} to {}, matchIndex: {}, writeIndex: {}", peerId, type.get(), target, matchIndex, writeIndex); + switch (target) { + case APPEND: + resetBatchAppendEntryRequest(); + break; + case COMPARE: + if (this.type.compareAndSet(EntryDispatcherState.APPEND, EntryDispatcherState.COMPARE)) { + writeIndex = dLedgerStore.getLedgerEndIndex() + 1; + pendingMap.clear(); + } + break; + default: + break; + } + type.set(target); + } + + @Override + public void doWork() { + try { + if (checkNotLeaderAndFreshState()) { + waitForRunning(1); + return; + } + switch (type.get()) { + case COMPARE: + doCompare(); + break; + case TRUNCATE: + doTruncate(); + break; + case APPEND: + doAppend(); + break; + case INSTALL_SNAPSHOT: + doInstallSnapshot(); + break; + case COMMIT: + doCommit(); + break; + } + waitForRunning(1); + } catch (Throwable t) { + DLedgerEntryPusher.LOGGER.error("[Push-{}]Error in {} writeIndex={} matchIndex={}", peerId, getName(), writeIndex, matchIndex, t); + changeState(EntryDispatcherState.COMPARE); + DLedgerUtils.sleep(500); + } + } + + /** + * First compare the leader store with the follower store, find the match index for the follower, and update write index to [matchIndex + 1] + * + * @throws Exception + */ + private void doCompare() throws Exception { + while (true) { + if (checkNotLeaderAndFreshState()) { + break; + } + if (this.type.get() != EntryDispatcherState.COMPARE) { + break; + } + if (dLedgerStore.getLedgerEndIndex() == -1) { + // now not entry in store + break; + } + + // compare process start from the [nextIndex -1] + PushEntryRequest request; + long compareIndex = writeIndex - 1; + long compareTerm = -1; + if (compareIndex < dLedgerStore.getLedgerBeforeBeginIndex()) { + // need compared entry has been dropped for compaction, just change state to install snapshot + changeState(EntryDispatcherState.INSTALL_SNAPSHOT); + return; + } else if (compareIndex == dLedgerStore.getLedgerBeforeBeginIndex()) { + compareTerm = dLedgerStore.getLedgerBeforeBeginTerm(); + request = buildCompareOrTruncatePushRequest(compareTerm, compareIndex, PushEntryRequest.Type.COMPARE); + } else { + DLedgerEntry entry = dLedgerStore.get(compareIndex); + PreConditions.check(entry != null, DLedgerResponseCode.INTERNAL_ERROR, "compareIndex=%d", compareIndex); + compareTerm = entry.getTerm(); + request = buildCompareOrTruncatePushRequest(compareTerm, entry.getIndex(), PushEntryRequest.Type.COMPARE); + } + CompletableFuture responseFuture = dLedgerRpcService.push(request); + PushEntryResponse response = responseFuture.get(3, TimeUnit.SECONDS); + PreConditions.check(response != null, DLedgerResponseCode.INTERNAL_ERROR, "compareIndex=%d", compareIndex); + PreConditions.check(response.getCode() == DLedgerResponseCode.INCONSISTENT_STATE.getCode() || response.getCode() == DLedgerResponseCode.SUCCESS.getCode() + , DLedgerResponseCode.valueOf(response.getCode()), "compareIndex=%d", compareIndex); + + // fast backup algorithm to locate the match index + if (response.getCode() == DLedgerResponseCode.SUCCESS.getCode()) { + // leader find the matched index for this follower + matchIndex = compareIndex; + updatePeerWaterMark(compareTerm, peerId, matchIndex); + // change state to truncate + changeState(EntryDispatcherState.TRUNCATE); + return; + } + + // inconsistent state, need to keep comparing + if (response.getXTerm() != -1) { + writeIndex = response.getXIndex(); + } else { + writeIndex = response.getEndIndex() + 1; + } } } + private void doTruncate() throws Exception { + PreConditions.check(type.get() == EntryDispatcherState.TRUNCATE, DLedgerResponseCode.UNKNOWN); + // truncate all entries after truncateIndex for follower + long truncateIndex = matchIndex + 1; + logger.info("[Push-{}]Will push data to truncate truncateIndex={}", peerId, truncateIndex); + PushEntryRequest truncateRequest = buildCompareOrTruncatePushRequest(-1, truncateIndex, PushEntryRequest.Type.TRUNCATE); + PushEntryResponse truncateResponse = dLedgerRpcService.push(truncateRequest).get(3, TimeUnit.SECONDS); + + PreConditions.check(truncateResponse != null, DLedgerResponseCode.UNKNOWN, "truncateIndex=%d", truncateIndex); + PreConditions.check(truncateResponse.getCode() == DLedgerResponseCode.SUCCESS.getCode(), DLedgerResponseCode.valueOf(truncateResponse.getCode()), "truncateIndex=%d", truncateIndex); + lastPushCommitTimeMs = System.currentTimeMillis(); + changeState(EntryDispatcherState.APPEND); + } + private void doAppend() throws Exception { while (true) { if (checkNotLeaderAndFreshState()) { break; } - if (type.get() != PushEntryRequest.Type.APPEND) { + if (type.get() != EntryDispatcherState.APPEND) { break; } + // check if first append request is timeout now + doCheckAppendResponse(); + // check if now not new entries to be sent if (writeIndex > dLedgerStore.getLedgerEndIndex()) { - doCommit(); - doCheckAppendResponse(); + if (this.batchAppendEntryRequest.getCount() > 0) { + sendBatchAppendEntryRequest(); + } else { + doCommit(); + } + break; + } + // check if now not entries in store can be sent + if (writeIndex <= dLedgerStore.getLedgerBeforeBeginIndex()) { + logger.info("[Push-{}]The ledgerBeginBeginIndex={} is less than or equal to writeIndex={}", peerId, dLedgerStore.getLedgerBeforeBeginIndex(), writeIndex); + changeState(EntryDispatcherState.INSTALL_SNAPSHOT); break; } if (pendingMap.size() >= maxPendingSize || DLedgerUtils.elapsed(lastCheckLeakTimeMs) > 1000) { long peerWaterMark = getPeerWaterMark(term, peerId); - for (Long index : pendingMap.keySet()) { - if (index < peerWaterMark) { - pendingMap.remove(index); + for (Map.Entry> entry : pendingMap.entrySet()) { + if (entry.getKey() + entry.getValue().getValue() - 1 <= peerWaterMark) { + // clear the append request which all entries have been accepted in peer + pendingMap.remove(entry.getKey()); } } lastCheckLeakTimeMs = System.currentTimeMillis(); @@ -543,264 +662,137 @@ private void doAppend() throws Exception { doCheckAppendResponse(); break; } - doAppendInner(writeIndex); - writeIndex++; + long lastIndexToBeSend = doAppendInner(writeIndex); + if (lastIndexToBeSend == -1) { + break; + } + writeIndex = lastIndexToBeSend + 1; } } + /** + * append the entries to the follower, append it in memory until the threshold is reached, its will be really sent to peer + * + * @param index from which index to append + * @return the index of the last entry to be appended + * @throws Exception + */ + private long doAppendInner(long index) throws Exception { + DLedgerEntry entry = getDLedgerEntryForAppend(index); + if (null == entry) { + // means should install snapshot + logger.error("[Push-{}]Get null entry from index={}", peerId, index); + changeState(EntryDispatcherState.INSTALL_SNAPSHOT); + return -1; + } + // check quota for flow controlling + checkQuotaAndWait(entry); + batchAppendEntryRequest.addEntry(entry); + // check if now can trigger real send + if (!dLedgerConfig.isEnableBatchAppend() || batchAppendEntryRequest.getTotalSize() >= dLedgerConfig.getMaxBatchAppendSize() + || DLedgerUtils.elapsed(this.lastAppendEntryRequestSendTimeMs) >= dLedgerConfig.getMaxBatchAppendIntervalMs()) { + sendBatchAppendEntryRequest(); + } + return entry.getIndex(); + } + private void sendBatchAppendEntryRequest() throws Exception { - batchAppendEntryRequest.setCommitIndex(dLedgerStore.getCommittedIndex()); + batchAppendEntryRequest.setCommitIndex(memberState.getCommittedIndex()); + final long firstIndex = batchAppendEntryRequest.getFirstEntryIndex(); + final long lastIndex = batchAppendEntryRequest.getLastEntryIndex(); + final long lastTerm = batchAppendEntryRequest.getLastEntryTerm(); CompletableFuture responseFuture = dLedgerRpcService.push(batchAppendEntryRequest); - batchPendingMap.put(batchAppendEntryRequest.getFirstEntryIndex(), new Pair<>(System.currentTimeMillis(), batchAppendEntryRequest.getCount())); + pendingMap.put(firstIndex, new Pair<>(System.currentTimeMillis(), batchAppendEntryRequest.getCount())); responseFuture.whenComplete((x, ex) -> { try { PreConditions.check(ex == null, DLedgerResponseCode.UNKNOWN); DLedgerResponseCode responseCode = DLedgerResponseCode.valueOf(x.getCode()); switch (responseCode) { case SUCCESS: - batchPendingMap.remove(x.getIndex()); - if (x.getCount() == 0) { - updatePeerWaterMark(x.getTerm(), peerId, x.getIndex()); - } else { - updatePeerWaterMark(x.getTerm(), peerId, x.getIndex() + x.getCount() - 1); + pendingMap.remove(firstIndex); + if (lastIndex > matchIndex) { + matchIndex = lastIndex; + updatePeerWaterMark(lastTerm, peerId, matchIndex); } break; case INCONSISTENT_STATE: - logger.info("[Push-{}]Get INCONSISTENT_STATE when batch push index={} term={}", peerId, x.getIndex(), x.getTerm()); - changeState(-1, PushEntryRequest.Type.COMPARE); + logger.info("[Push-{}]Get INCONSISTENT_STATE when append entries from {} to {} when term is {}", peerId, firstIndex, lastIndex, term); + changeState(EntryDispatcherState.COMPARE); break; default: logger.warn("[Push-{}]Get error response code {} {}", peerId, responseCode, x.baseInfo()); break; } } catch (Throwable t) { - logger.error("", t); + logger.error("Failed to deal with the callback when append request return", t); } }); lastPushCommitTimeMs = System.currentTimeMillis(); batchAppendEntryRequest.clear(); } - private void doBatchAppendInner(long index) throws Exception { - DLedgerEntry entry = getDLedgerEntryForAppend(index); - if (null == entry) { + private void doInstallSnapshot() throws Exception { + // get snapshot from snapshot manager + if (checkNotLeaderAndFreshState()) { return; } - checkQuotaAndWait(entry); - batchAppendEntryRequest.addEntry(entry); - if (batchAppendEntryRequest.getTotalSize() >= dLedgerConfig.getMaxBatchPushSize()) { - sendBatchAppendEntryRequest(); + if (type.get() != EntryDispatcherState.INSTALL_SNAPSHOT) { + return; } - } - - private void doCheckBatchAppendResponse() throws Exception { - long peerWaterMark = getPeerWaterMark(term, peerId); - Pair pair = batchPendingMap.get(peerWaterMark + 1); - if (pair != null && System.currentTimeMillis() - pair.getKey() > dLedgerConfig.getMaxPushTimeOutMs()) { - long firstIndex = peerWaterMark + 1; - long lastIndex = firstIndex + pair.getValue() - 1; - logger.warn("[Push-{}]Retry to push entry from {} to {}", peerId, firstIndex, lastIndex); - batchAppendEntryRequest.clear(); - for (long i = firstIndex; i <= lastIndex; i++) { - DLedgerEntry entry = dLedgerStore.get(i); - batchAppendEntryRequest.addEntry(entry); - } - sendBatchAppendEntryRequest(); + if (fsmCaller.getSnapshotManager() == null) { + logger.error("[DoInstallSnapshot-{}]snapshot mode is disabled", peerId); + changeState(EntryDispatcherState.COMPARE); + return; } - } - - private void doBatchAppend() throws Exception { - while (true) { - if (checkNotLeaderAndFreshState()) { - break; - } - if (type.get() != PushEntryRequest.Type.APPEND) { - break; - } - if (writeIndex > dLedgerStore.getLedgerEndIndex()) { - if (batchAppendEntryRequest.getCount() > 0) { - sendBatchAppendEntryRequest(); - } - doCommit(); - doCheckBatchAppendResponse(); - break; - } - if (batchPendingMap.size() >= maxPendingSize || DLedgerUtils.elapsed(lastCheckLeakTimeMs) > 1000) { - long peerWaterMark = getPeerWaterMark(term, peerId); - for (Map.Entry> entry : batchPendingMap.entrySet()) { - if (entry.getKey() + entry.getValue().getValue() - 1 <= peerWaterMark) { - batchPendingMap.remove(entry.getKey()); - } - } - lastCheckLeakTimeMs = System.currentTimeMillis(); - } - if (batchPendingMap.size() >= maxPendingSize) { - doCheckBatchAppendResponse(); - break; - } - doBatchAppendInner(writeIndex); - writeIndex++; + SnapshotManager manager = fsmCaller.getSnapshotManager(); + SnapshotReader snpReader = manager.getSnapshotReaderIncludedTargetIndex(writeIndex); + if (snpReader == null) { + logger.error("[DoInstallSnapshot-{}]get latest snapshot whose lastIncludedIndex >= {} failed", peerId, writeIndex); + changeState(EntryDispatcherState.COMPARE); + return; } - } - - private void doTruncate(long truncateIndex) throws Exception { - PreConditions.check(type.get() == PushEntryRequest.Type.TRUNCATE, DLedgerResponseCode.UNKNOWN); - DLedgerEntry truncateEntry = dLedgerStore.get(truncateIndex); - PreConditions.check(truncateEntry != null, DLedgerResponseCode.UNKNOWN); - - logger.info("[Push-{}]Will push data to truncate truncateIndex={} pos={}", peerId, truncateIndex, truncateEntry.getPos()); - PushEntryRequest truncateRequest = buildPushRequest(truncateEntry, PushEntryRequest.Type.TRUNCATE); - PushEntryResponse truncateResponse = dLedgerRpcService.push(truncateRequest).get(3, TimeUnit.SECONDS); - PreConditions.check(truncateResponse != null, DLedgerResponseCode.UNKNOWN, "truncateIndex=%d", truncateIndex); - - PreConditions.check(truncateResponse.getCode() == DLedgerResponseCode.SUCCESS.getCode(), DLedgerResponseCode.valueOf(truncateResponse.getCode()), "truncateIndex=%d", truncateIndex); - lastPushCommitTimeMs = System.currentTimeMillis(); - changeState(truncateIndex, PushEntryRequest.Type.APPEND); - } - - private synchronized void changeState(long index, PushEntryRequest.Type target) { - logger.info("[Push-{}]Change state from {} to {} at {}", peerId, type.get(), target, index); - switch (target) { - case APPEND: - compareIndex = -1; - updatePeerWaterMark(term, peerId, index); - quorumAckChecker.wakeup(); - writeIndex = index + 1; - if (dLedgerConfig.isEnableBatchPush()) { - resetBatchAppendEntryRequest(); - } - break; - case COMPARE: - if (this.type.compareAndSet(PushEntryRequest.Type.APPEND, PushEntryRequest.Type.COMPARE)) { - compareIndex = -1; - if (dLedgerConfig.isEnableBatchPush()) { - batchPendingMap.clear(); - } else { - pendingMap.clear(); - } + DownloadSnapshot snapshot = snpReader.generateDownloadSnapshot(); + if (snapshot == null) { + logger.error("[DoInstallSnapshot-{}]generate latest snapshot for download failed, index = {}", peerId, writeIndex); + changeState(EntryDispatcherState.COMPARE); + return; + } + long lastIncludedIndex = snapshot.getMeta().getLastIncludedIndex(); + long lastIncludedTerm = snapshot.getMeta().getLastIncludedTerm(); + InstallSnapshotRequest request = buildInstallSnapshotRequest(snapshot); + CompletableFuture future = DLedgerEntryPusher.this.dLedgerRpcService.installSnapshot(request); + InstallSnapshotResponse response = future.get(3, TimeUnit.SECONDS); + PreConditions.check(response != null, DLedgerResponseCode.INTERNAL_ERROR, "installSnapshot lastIncludedIndex=%d", writeIndex); + DLedgerResponseCode responseCode = DLedgerResponseCode.valueOf(response.getCode()); + switch (responseCode) { + case SUCCESS: + logger.info("[DoInstallSnapshot-{}]install snapshot success, lastIncludedIndex = {}, lastIncludedTerm", peerId, lastIncludedIndex, lastIncludedTerm); + if (lastIncludedIndex > matchIndex) { + matchIndex = lastIncludedIndex; + writeIndex = matchIndex + 1; } + changeState(EntryDispatcherState.APPEND); break; - case TRUNCATE: - compareIndex = -1; + case INSTALL_SNAPSHOT_ERROR: + case INCONSISTENT_STATE: + logger.info("[DoInstallSnapshot-{}]install snapshot failed, index = {}, term = {}", peerId, writeIndex, term); + changeState(EntryDispatcherState.COMPARE); break; default: + logger.warn("[DoInstallSnapshot-{}]install snapshot failed because error response: code = {}, mas = {}, index = {}, term = {}", peerId, responseCode, response.baseInfo(), writeIndex, term); + changeState(EntryDispatcherState.COMPARE); break; } - type.set(target); } - private void doCompare() throws Exception { - while (true) { - if (checkNotLeaderAndFreshState()) { - break; - } - if (type.get() != PushEntryRequest.Type.COMPARE - && type.get() != PushEntryRequest.Type.TRUNCATE) { - break; - } - if (compareIndex == -1 && dLedgerStore.getLedgerEndIndex() == -1) { - break; - } - //revise the compareIndex - if (compareIndex == -1) { - compareIndex = dLedgerStore.getLedgerEndIndex(); - logger.info("[Push-{}][DoCompare] compareIndex=-1 means start to compare", peerId); - } else if (compareIndex > dLedgerStore.getLedgerEndIndex() || compareIndex <= dLedgerStore.getLedgerBeforeBeginIndex()) { - logger.info("[Push-{}][DoCompare] compareIndex={} out of range ({}-{}]", peerId, compareIndex, dLedgerStore.getLedgerBeforeBeginIndex(), dLedgerStore.getLedgerEndIndex()); - compareIndex = dLedgerStore.getLedgerEndIndex(); - } - - DLedgerEntry entry = dLedgerStore.get(compareIndex); - PreConditions.check(entry != null, DLedgerResponseCode.INTERNAL_ERROR, "compareIndex=%d", compareIndex); - PushEntryRequest request = buildPushRequest(entry, PushEntryRequest.Type.COMPARE); - CompletableFuture responseFuture = dLedgerRpcService.push(request); - PushEntryResponse response = responseFuture.get(3, TimeUnit.SECONDS); - PreConditions.check(response != null, DLedgerResponseCode.INTERNAL_ERROR, "compareIndex=%d", compareIndex); - - PreConditions.check(response.getCode() == DLedgerResponseCode.INCONSISTENT_STATE.getCode() || response.getCode() == DLedgerResponseCode.SUCCESS.getCode() - , DLedgerResponseCode.valueOf(response.getCode()), "compareIndex=%d", compareIndex); - long truncateIndex = -1; - - if (response.getCode() == DLedgerResponseCode.SUCCESS.getCode()) { - /* - * The comparison is successful: - * 1.Just change to append state, if the follower's end index is equal the compared index. - * 2.Truncate the follower, if the follower has some dirty entries. - */ - if (compareIndex == response.getEndIndex()) { - changeState(compareIndex, PushEntryRequest.Type.APPEND); - break; - } else { - truncateIndex = compareIndex; - } - } else if (response.getEndIndex() <= dLedgerStore.getLedgerBeforeBeginIndex() - || response.getBeginIndex() > dLedgerStore.getLedgerEndIndex()) { - /* - The follower's entries does not intersect with the leader. - This usually happened when the follower has crashed for a long time while the leader has deleted the expired entries. - Just truncate the follower. - */ - truncateIndex = dLedgerStore.getLedgerBeforeBeginIndex() + 1; - } else if (compareIndex < response.getBeginIndex()) { - /* - The compared index is smaller than the follower's begin index. - This happened rarely, usually means some disk damage. - Just truncate the follower. - */ - truncateIndex = dLedgerStore.getLedgerBeforeBeginIndex() + 1; - } else if (compareIndex > response.getEndIndex()) { - /* - The compared index is bigger than the follower's end index. - This happened frequently. For the compared index is usually starting from the end index of the leader. - */ - compareIndex = response.getEndIndex(); - } else { - /* - Compare failed and the compared index is in the range of follower's entries. - */ - compareIndex--; - } - /* - The compared index is smaller than the leader's begin index, truncate the follower. - */ - if (compareIndex <= dLedgerStore.getLedgerBeforeBeginIndex()) { - truncateIndex = dLedgerStore.getLedgerBeforeBeginIndex() + 1; - } - /* - If get value for truncateIndex, do it right now. - */ - if (truncateIndex != -1) { - changeState(truncateIndex, PushEntryRequest.Type.TRUNCATE); - doTruncate(truncateIndex); - break; - } - } - } - - @Override - public void doWork() { - try { - if (checkNotLeaderAndFreshState()) { - waitForRunning(1); - return; - } + } - if (type.get() == PushEntryRequest.Type.APPEND) { - if (dLedgerConfig.isEnableBatchPush()) { - doBatchAppend(); - } else { - doAppend(); - } - } else { - doCompare(); - } - waitForRunning(1); - } catch (Throwable t) { - DLedgerEntryPusher.LOGGER.error("[Push-{}]Error in {} writeIndex={} compareIndex={}", peerId, getName(), writeIndex, compareIndex, t); - changeState(-1, PushEntryRequest.Type.COMPARE); - DLedgerUtils.sleep(500); - } - } + enum EntryDispatcherState { + COMPARE, + TRUNCATE, + APPEND, + INSTALL_SNAPSHOT, + COMMIT } /** @@ -811,24 +803,54 @@ private class EntryHandler extends ShutdownAbleThread { private long lastCheckFastForwardTimeMs = System.currentTimeMillis(); - ConcurrentMap>> writeRequestMap = new ConcurrentHashMap<>(); + ConcurrentMap>> writeRequestMap = new ConcurrentHashMap<>(); BlockingQueue>> compareOrTruncateRequests = new ArrayBlockingQueue<>(1024); + private ReentrantLock inflightInstallSnapshotRequestLock = new ReentrantLock(); + + private Pair> inflightInstallSnapshotRequest; + public EntryHandler(Logger logger) { super("EntryHandler-" + memberState.getSelfId(), logger); } + public CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) { + CompletableFuture future = new TimeoutFuture<>(1000); + PreConditions.check(request.getData() != null && request.getData().length > 0, DLedgerResponseCode.UNEXPECTED_ARGUMENT); + long index = request.getLastIncludedIndex(); + inflightInstallSnapshotRequestLock.lock(); + try { + CompletableFuture oldFuture = null; + if (inflightInstallSnapshotRequest != null && inflightInstallSnapshotRequest.getKey().getLastIncludedIndex() >= index) { + oldFuture = future; + logger.warn("[MONITOR]The install snapshot request with index {} has already existed", index, inflightInstallSnapshotRequest.getKey()); + } else { + logger.warn("[MONITOR]The install snapshot request with index {} preempt inflight slot because of newer index", index); + if (inflightInstallSnapshotRequest != null && inflightInstallSnapshotRequest.getValue() != null) { + oldFuture = inflightInstallSnapshotRequest.getValue(); + } + inflightInstallSnapshotRequest = new Pair<>(request, future); + } + if (oldFuture != null) { + InstallSnapshotResponse response = new InstallSnapshotResponse(); + response.setGroup(request.getGroup()); + response.setCode(DLedgerResponseCode.NEWER_INSTALL_SNAPSHOT_REQUEST_EXIST.getCode()); + response.setTerm(request.getTerm()); + oldFuture.complete(response); + } + } finally { + inflightInstallSnapshotRequestLock.unlock(); + } + return future; + } + public CompletableFuture handlePush(PushEntryRequest request) throws Exception { - //The timeout should smaller than the remoting layer's request timeout + // The timeout should smaller than the remoting layer's request timeout CompletableFuture future = new TimeoutFuture<>(1000); switch (request.getType()) { case APPEND: - if (request.isBatch()) { - PreConditions.check(request.getBatchEntry() != null && request.getCount() > 0, DLedgerResponseCode.UNEXPECTED_ARGUMENT); - } else { - PreConditions.check(request.getEntry() != null, DLedgerResponseCode.UNEXPECTED_ARGUMENT); - } + PreConditions.check(request.getCount() > 0, DLedgerResponseCode.UNEXPECTED_ARGUMENT); long index = request.getFirstEntryIndex(); Pair> old = writeRequestMap.putIfAbsent(index, new Pair<>(request, future)); if (old != null) { @@ -846,7 +868,6 @@ public CompletableFuture handlePush(PushEntryRequest request) break; case COMPARE: case TRUNCATE: - PreConditions.check(request.getEntry() != null, DLedgerResponseCode.UNEXPECTED_ARGUMENT); writeRequestMap.clear(); synchronized (this) { if (!compareOrTruncateRequests.offer(new Pair<>(request, future))) { @@ -873,46 +894,84 @@ private PushEntryResponse buildResponse(PushEntryRequest request, int code) { response.setIndex(request.getFirstEntryIndex()); response.setCount(request.getCount()); } - response.setBeginIndex(dLedgerStore.getLedgerBeforeBeginIndex() + 1); - response.setEndIndex(dLedgerStore.getLedgerEndIndex()); return response; } private void handleDoAppend(long writeIndex, PushEntryRequest request, - CompletableFuture future) { + CompletableFuture future) { try { - PreConditions.check(writeIndex == request.getEntry().getIndex(), DLedgerResponseCode.INCONSISTENT_STATE); - DLedgerEntry entry = dLedgerStore.appendAsFollower(request.getEntry(), request.getTerm(), request.getLeaderId()); - PreConditions.check(entry.getIndex() == writeIndex, DLedgerResponseCode.INCONSISTENT_STATE); + PreConditions.check(writeIndex == request.getFirstEntryIndex(), DLedgerResponseCode.INCONSISTENT_STATE); + for (DLedgerEntry entry : request.getEntries()) { + dLedgerStore.appendAsFollower(entry, request.getTerm(), request.getLeaderId()); + } future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); - updateCommittedIndex(request.getTerm(), request.getCommitIndex()); + long committedIndex = Math.min(dLedgerStore.getLedgerEndIndex(), request.getCommitIndex()); + if (DLedgerEntryPusher.this.memberState.followerUpdateCommittedIndex(committedIndex)) { + DLedgerEntryPusher.this.fsmCaller.onCommitted(committedIndex); + } } catch (Throwable t) { logger.error("[HandleDoAppend] writeIndex={}", writeIndex, t); future.complete(buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode())); } } - private CompletableFuture handleDoCompare(long compareIndex, PushEntryRequest request, - CompletableFuture future) { + private CompletableFuture handleDoCompare(PushEntryRequest request, + CompletableFuture future) { try { - PreConditions.check(compareIndex == request.getEntry().getIndex(), DLedgerResponseCode.UNKNOWN); PreConditions.check(request.getType() == PushEntryRequest.Type.COMPARE, DLedgerResponseCode.UNKNOWN); - DLedgerEntry local = dLedgerStore.get(compareIndex); - PreConditions.check(request.getEntry().equals(local), DLedgerResponseCode.INCONSISTENT_STATE); - future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); + // fast backup algorithm + long preLogIndex = request.getPreLogIndex(); + long preLogTerm = request.getPreLogTerm(); + if (preLogTerm == -1 && preLogIndex == -1) { + // leader's entries is empty + future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); + return future; + } + if (dLedgerStore.getLedgerEndIndex() >= preLogIndex) { + long compareTerm = 0; + if (dLedgerStore.getLedgerBeforeBeginIndex() == preLogIndex) { + // the preLogIndex is smaller than the smallest index of the ledger, so just compare the snapshot last included term + compareTerm = dLedgerStore.getLedgerBeforeBeginTerm(); + } else { + // there exist a log whose index is preLogIndex + DLedgerEntry local = dLedgerStore.get(preLogIndex); + compareTerm = local.getTerm(); + } + if (compareTerm == preLogTerm) { + // the log's term is preLogTerm + // all matched! + future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); + return future; + } + // if the log's term is not preLogTerm, we need to find the first log of this term + DLedgerEntry firstEntryWithTargetTerm = dLedgerStore.getFirstLogOfTargetTerm(compareTerm, preLogIndex); + PreConditions.check(firstEntryWithTargetTerm != null, DLedgerResponseCode.INCONSISTENT_STATE); + PushEntryResponse response = buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode()); + response.setXTerm(compareTerm); + response.setXIndex(firstEntryWithTargetTerm.getIndex()); + future.complete(response); + return future; + } + // if there doesn't exist entry in preLogIndex, we return last entry index + PushEntryResponse response = buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode()); + response.setEndIndex(dLedgerStore.getLedgerEndIndex()); + future.complete(response); } catch (Throwable t) { - logger.error("[HandleDoCompare] compareIndex={}", compareIndex, t); + logger.error("[HandleDoCompare] preLogIndex={}, preLogTerm={}", request.getPreLogIndex(), request.getPreLogTerm(), t); future.complete(buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode())); } return future; } private CompletableFuture handleDoCommit(long committedIndex, PushEntryRequest request, - CompletableFuture future) { + CompletableFuture future) { try { PreConditions.check(committedIndex == request.getCommitIndex(), DLedgerResponseCode.UNKNOWN); PreConditions.check(request.getType() == PushEntryRequest.Type.COMMIT, DLedgerResponseCode.UNKNOWN); - updateCommittedIndex(request.getTerm(), committedIndex); + committedIndex = committedIndex <= dLedgerStore.getLedgerEndIndex() ? committedIndex : dLedgerStore.getLedgerEndIndex(); + if (DLedgerEntryPusher.this.memberState.followerUpdateCommittedIndex(committedIndex)) { + DLedgerEntryPusher.this.fsmCaller.onCommitted(committedIndex); + } future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); } catch (Throwable t) { logger.error("[HandleDoCommit] committedIndex={}", request.getCommitIndex(), t); @@ -922,15 +981,17 @@ private CompletableFuture handleDoCommit(long committedIndex, } private CompletableFuture handleDoTruncate(long truncateIndex, PushEntryRequest request, - CompletableFuture future) { + CompletableFuture future) { try { - logger.info("[HandleDoTruncate] truncateIndex={} pos={}", truncateIndex, request.getEntry().getPos()); - PreConditions.check(truncateIndex == request.getEntry().getIndex(), DLedgerResponseCode.UNKNOWN); + logger.info("[HandleDoTruncate] truncateIndex={}", truncateIndex); PreConditions.check(request.getType() == PushEntryRequest.Type.TRUNCATE, DLedgerResponseCode.UNKNOWN); - long index = dLedgerStore.truncate(request.getEntry(), request.getTerm(), request.getLeaderId()); - PreConditions.check(index == truncateIndex, DLedgerResponseCode.INCONSISTENT_STATE); + long index = dLedgerStore.truncate(truncateIndex); + PreConditions.check(index == truncateIndex - 1, DLedgerResponseCode.INCONSISTENT_STATE); future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); - updateCommittedIndex(request.getTerm(), request.getCommitIndex()); + long committedIndex = request.getCommitIndex() <= dLedgerStore.getLedgerEndIndex() ? request.getCommitIndex() : dLedgerStore.getLedgerEndIndex(); + if (DLedgerEntryPusher.this.memberState.followerUpdateCommittedIndex(committedIndex)) { + DLedgerEntryPusher.this.fsmCaller.onCommitted(committedIndex); + } } catch (Throwable t) { logger.error("[HandleDoTruncate] truncateIndex={}", truncateIndex, t); future.complete(buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode())); @@ -938,20 +999,26 @@ private CompletableFuture handleDoTruncate(long truncateIndex return future; } - private void handleDoBatchAppend(long writeIndex, PushEntryRequest request, - CompletableFuture future) { + private void handleDoInstallSnapshot(InstallSnapshotRequest request, + CompletableFuture future) { + InstallSnapshotResponse response = new InstallSnapshotResponse(); + response.setGroup(request.getGroup()); + response.copyBaseInfo(request); try { - PreConditions.check(writeIndex == request.getFirstEntryIndex(), DLedgerResponseCode.INCONSISTENT_STATE); - for (DLedgerEntry entry : request.getBatchEntry()) { - dLedgerStore.appendAsFollower(entry, request.getTerm(), request.getLeaderId()); + logger.info("[HandleDoInstallSnapshot] begin to install snapshot, request={}", request); + DownloadSnapshot snapshot = new DownloadSnapshot(new SnapshotMeta(request.getLastIncludedIndex(), request.getLastIncludedTerm()), request.getData()); + if (!fsmCaller.getSnapshotManager().installSnapshot(snapshot)) { + response.code(DLedgerResponseCode.INSTALL_SNAPSHOT_ERROR.getCode()); + future.complete(response); + return; } - future.complete(buildResponse(request, DLedgerResponseCode.SUCCESS.getCode())); - updateCommittedIndex(request.getTerm(), request.getCommitIndex()); + response.code(DLedgerResponseCode.SUCCESS.getCode()); + future.complete(response); } catch (Throwable t) { - logger.error("[HandleDoBatchAppend]", t); - future.complete(buildResponse(request, DLedgerResponseCode.INCONSISTENT_STATE.getCode())); + logger.error("[HandleDoInstallSnapshot] install snapshot failed, request={}", request, t); + response.code(DLedgerResponseCode.INSTALL_SNAPSHOT_ERROR.getCode()); + future.complete(response); } - } private void checkAppendFuture(long endIndex) { @@ -959,15 +1026,10 @@ private void checkAppendFuture(long endIndex) { for (Pair> pair : writeRequestMap.values()) { long firstEntryIndex = pair.getKey().getFirstEntryIndex(); long lastEntryIndex = pair.getKey().getLastEntryIndex(); - //Fall behind + // clear old push request if (lastEntryIndex <= endIndex) { try { - if (pair.getKey().isBatch()) { - for (DLedgerEntry dLedgerEntry : pair.getKey().getBatchEntry()) { - PreConditions.check(dLedgerEntry.equals(dLedgerStore.get(dLedgerEntry.getIndex())), DLedgerResponseCode.INCONSISTENT_STATE); - } - } else { - DLedgerEntry dLedgerEntry = pair.getKey().getEntry(); + for (DLedgerEntry dLedgerEntry : pair.getKey().getEntries()) { PreConditions.check(dLedgerEntry.equals(dLedgerStore.get(dLedgerEntry.getIndex())), DLedgerResponseCode.INCONSISTENT_STATE); } pair.getValue().complete(buildResponse(pair.getKey(), DLedgerResponseCode.SUCCESS.getCode())); @@ -979,9 +1041,11 @@ private void checkAppendFuture(long endIndex) { writeRequestMap.remove(pair.getKey().getFirstEntryIndex()); continue; } + // normal case if (firstEntryIndex == endIndex + 1) { return; } + // clear timeout push request TimeoutFuture future = (TimeoutFuture) pair.getValue(); if (!future.isTimeOut()) { continue; @@ -1040,15 +1104,30 @@ public void doWork() { waitForRunning(1); return; } + // deal with install snapshot request first + Pair> installSnapshotPair = null; + this.inflightInstallSnapshotRequestLock.lock(); + try { + if (inflightInstallSnapshotRequest != null && inflightInstallSnapshotRequest.getKey() != null && inflightInstallSnapshotRequest.getValue() != null) { + installSnapshotPair = inflightInstallSnapshotRequest; + inflightInstallSnapshotRequest = new Pair<>(null, null); + } + } finally { + this.inflightInstallSnapshotRequestLock.unlock(); + } + if (installSnapshotPair != null) { + handleDoInstallSnapshot(installSnapshotPair.getKey(), installSnapshotPair.getValue()); + } + // deal with the compare or truncate requests if (compareOrTruncateRequests.peek() != null) { Pair> pair = compareOrTruncateRequests.poll(); PreConditions.check(pair != null, DLedgerResponseCode.UNKNOWN); switch (pair.getKey().getType()) { case TRUNCATE: - handleDoTruncate(pair.getKey().getEntry().getIndex(), pair.getKey(), pair.getValue()); + handleDoTruncate(pair.getKey().getPreLogIndex(), pair.getKey(), pair.getValue()); break; case COMPARE: - handleDoCompare(pair.getKey().getEntry().getIndex(), pair.getKey(), pair.getValue()); + handleDoCompare(pair.getKey(), pair.getValue()); break; case COMMIT: handleDoCommit(pair.getKey().getCommitIndex(), pair.getKey(), pair.getValue()); @@ -1056,21 +1135,17 @@ public void doWork() { default: break; } - } else { - long nextIndex = dLedgerStore.getLedgerEndIndex() + 1; - Pair> pair = writeRequestMap.remove(nextIndex); - if (pair == null) { - checkAbnormalFuture(dLedgerStore.getLedgerEndIndex()); - waitForRunning(1); - return; - } - PushEntryRequest request = pair.getKey(); - if (request.isBatch()) { - handleDoBatchAppend(nextIndex, request, pair.getValue()); - } else { - handleDoAppend(nextIndex, request, pair.getValue()); - } + return; + } + long nextIndex = dLedgerStore.getLedgerEndIndex() + 1; + Pair> pair = writeRequestMap.remove(nextIndex); + if (pair == null) { + checkAbnormalFuture(dLedgerStore.getLedgerEndIndex()); + waitForRunning(1); + return; } + PushEntryRequest request = pair.getKey(); + handleDoAppend(nextIndex, request, pair.getValue()); } catch (Throwable t) { DLedgerEntryPusher.LOGGER.error("Error in {}", getName(), t); DLedgerUtils.sleep(100); diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java index 6c90ed28..e23bff61 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerLeaderElector.java @@ -17,6 +17,7 @@ package io.openmessaging.storage.dledger; import com.alibaba.fastjson.JSON; +import io.openmessaging.storage.dledger.common.ShutdownAbleThread; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; @@ -68,7 +69,7 @@ public class DLedgerLeaderElector { private VoteResponse.ParseResult lastParseResult = VoteResponse.ParseResult.WAIT_TO_REVOTE; private long lastVoteCost = 0L; - private final StateMaintainer stateMaintainer = new StateMaintainer("StateMaintainer", LOGGER); + private final StateMaintainer stateMaintainer; private final TakeLeadershipTask takeLeadershipTask = new TakeLeadershipTask(); @@ -77,6 +78,7 @@ public DLedgerLeaderElector(DLedgerConfig dLedgerConfig, MemberState memberState this.dLedgerConfig = dLedgerConfig; this.memberState = memberState; this.dLedgerRpcService = dLedgerRpcService; + this.stateMaintainer = new StateMaintainer("StateMaintainer-" + this.memberState.getSelfId(), LOGGER); refreshIntervals(dLedgerConfig); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerRpcNettyService.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerRpcNettyService.java index 79185991..97e51b0d 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerRpcNettyService.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerRpcNettyService.java @@ -18,6 +18,7 @@ import com.alibaba.fastjson.JSON; import io.netty.channel.ChannelHandlerContext; +import io.openmessaging.storage.dledger.common.NamedThreadFactory; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.DLedgerRequestCode; @@ -26,6 +27,8 @@ import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.LeadershipTransferRequest; import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; import io.openmessaging.storage.dledger.protocol.MetadataRequest; @@ -73,7 +76,7 @@ public class DLedgerRpcNettyService extends DLedgerRpcService { private AbstractDLedgerServer dLedger; - private final ConcurrentHashMap> userDefineProcessors = new ConcurrentHashMap>(); + private final ConcurrentHashMap> userDefineProcessors = new ConcurrentHashMap>(); private final ExecutorService futureExecutor = Executors.newFixedThreadPool(4, new NamedThreadFactory("FutureExecutor")); @@ -118,7 +121,7 @@ public boolean rejectRequest() { } @Override - public void registerUserDefineProcessor(UserDefineProcessor userDefineProcessor) { + public void registerUserDefineProcessor(UserDefineProcessor userDefineProcessor) { this.userDefineProcessors.put(userDefineProcessor.getRequestTypeCode(), userDefineProcessor); } @@ -132,6 +135,7 @@ private void registerProcessor(NettyRemotingServer remotingServer, NettyRequestP remotingServer.registerProcessor(DLedgerRequestCode.HEART_BEAT.getCode(), protocolProcessor, null); remotingServer.registerProcessor(DLedgerRequestCode.LEADERSHIP_TRANSFER.getCode(), protocolProcessor, null); remotingServer.registerProcessor(DLedgerRequestCode.USER_DEFINE_REQUEST.getCode(), protocolProcessor, null); + remotingServer.registerProcessor(DLedgerRequestCode.INSTALL_SNAPSHOT.getCode(), protocolProcessor, null); } private NettyRemotingServer registerRemotingServer(NettyServerConfig nettyServerConfig, ChannelEventListener channelEventListener, NettyRequestProcessor protocolProcessor) { @@ -277,6 +281,35 @@ public CompletableFuture push(PushEntryRequest request) { return future; } + @Override + public CompletableFuture installSnapshot(InstallSnapshotRequest request) { + CompletableFuture future = new CompletableFuture<>(); + try { + RemotingCommand wrapperRequest = RemotingCommand.createRequestCommand(DLedgerRequestCode.INSTALL_SNAPSHOT.getCode(), null); + wrapperRequest.setBody(JSON.toJSONBytes(request)); + remotingClient.invokeAsync(getPeerAddr(request.getGroup(), request.getRemoteId()), wrapperRequest, 3000, responseFuture -> { + RemotingCommand responseCommand = responseFuture.getResponseCommand(); + + InstallSnapshotResponse response; + if (responseCommand != null) { + response = JSON.parseObject(responseFuture.getResponseCommand().getBody(), InstallSnapshotResponse.class); + } else { + response = new InstallSnapshotResponse(); + response.copyBaseInfo(request); + response.setCode(DLedgerResponseCode.NETWORK_ERROR.getCode()); + } + future.complete(response); + }); + } catch (Throwable t) { + LOGGER.error("Send install snapshot request failed, {}", request.baseInfo(), t); + InstallSnapshotResponse response = new InstallSnapshotResponse(); + response.copyBaseInfo(request); + response.setCode(DLedgerResponseCode.NETWORK_ERROR.getCode()); + future.complete(response); + } + return future; + } + @Override public CompletableFuture leadershipTransfer( LeadershipTransferRequest request) { @@ -395,6 +428,14 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand }, futureExecutor); break; } + case INSTALL_SNAPSHOT: { + InstallSnapshotRequest installSnapshotRequest = JSON.parseObject(request.getBody(), InstallSnapshotRequest.class); + CompletableFuture future = handleInstallSnapshot(installSnapshotRequest); + future.whenCompleteAsync((x, y) -> { + writeResponse(x, y, request, ctx); + }, futureExecutor); + break; + } case USER_DEFINE_REQUEST: UserDefineCommandHeader header = (UserDefineCommandHeader) request.decodeCommandCustomHeader(UserDefineCommandHeader.class); UserDefineProcessor userDefineProcessor = this.userDefineProcessors.get(header.getRequestTypeCode()); @@ -453,6 +494,11 @@ public CompletableFuture handlePush(PushEntryRequest request) return this.dLedger.handlePush(request); } + @Override + public CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) throws Exception { + return this.dLedger.handleInstallSnapshot(request); + } + public RemotingCommand handleResponse(RequestOrResponse response, RemotingCommand request) { RemotingCommand remotingCommand = RemotingCommand.createResponseCommand(DLedgerResponseCode.SUCCESS.getCode(), null); remotingCommand.setBody(JSON.toJSONBytes(response)); diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerServer.java b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerServer.java index bb98d485..2a575803 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerServer.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerServer.java @@ -16,6 +16,13 @@ package io.openmessaging.storage.dledger; +import io.openmessaging.storage.dledger.common.AppendFuture; +import io.openmessaging.storage.dledger.common.BatchAppendFuture; +import io.openmessaging.storage.dledger.common.Closure; +import io.openmessaging.storage.dledger.common.NamedThreadFactory; +import io.openmessaging.storage.dledger.common.ReadClosure; +import io.openmessaging.storage.dledger.common.ReadMode; +import io.openmessaging.storage.dledger.common.Status; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.exception.DLedgerException; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; @@ -26,6 +33,8 @@ import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.LeadershipTransferRequest; import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; import io.openmessaging.storage.dledger.protocol.MetadataRequest; @@ -38,6 +47,7 @@ import io.openmessaging.storage.dledger.protocol.VoteResponse; import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; import io.openmessaging.storage.dledger.snapshot.SnapshotManager; +import io.openmessaging.storage.dledger.statemachine.NoOpStatemachine; import io.openmessaging.storage.dledger.statemachine.StateMachine; import io.openmessaging.storage.dledger.statemachine.StateMachineCaller; import io.openmessaging.storage.dledger.store.DLedgerMemoryStore; @@ -53,6 +63,7 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -81,16 +92,17 @@ public class DLedgerServer extends AbstractDLedgerServer { private final DLedgerLeaderElector dLedgerLeaderElector; private final ScheduledExecutorService executorService; - private Optional fsmCaller; + + private StateMachineCaller fsmCaller; private volatile boolean isStarted = false; public DLedgerServer(DLedgerConfig dLedgerConfig) { - this(dLedgerConfig, null, null, null); + this(dLedgerConfig, null, null); } public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerConfig) { - this(dLedgerConfig, nettyServerConfig, null, null); + this(dLedgerConfig, nettyServerConfig, null); } public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerConfig, @@ -100,6 +112,11 @@ public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerC public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerConfig, NettyClientConfig nettyClientConfig, ChannelEventListener channelEventListener) { + this(dLedgerConfig, nettyServerConfig, nettyClientConfig, channelEventListener, null); + } + + public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerConfig, + NettyClientConfig nettyClientConfig, ChannelEventListener channelEventListener, StateMachine stateMachine) { dLedgerConfig.init(); this.dLedgerConfig = dLedgerConfig; this.memberState = new MemberState(dLedgerConfig); @@ -109,7 +126,10 @@ public DLedgerServer(DLedgerConfig dLedgerConfig, NettyServerConfig nettyServerC this.dLedgerEntryPusher = new DLedgerEntryPusher(dLedgerConfig, memberState, dLedgerStore, dLedgerRpcService); this.dLedgerLeaderElector = new DLedgerLeaderElector(dLedgerConfig, memberState, dLedgerRpcService); this.executorService = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory(null, "DLedgerServer-ScheduledExecutor", true)); - this.fsmCaller = Optional.empty(); + if (stateMachine == null) { + stateMachine = new NoOpStatemachine(); + } + this.fsmCaller = new StateMachineCaller(this.dLedgerStore, stateMachine, this.dLedgerEntryPusher); } /** @@ -132,7 +152,7 @@ public DLedgerServer(DLedgerConfig dLedgerConfig, DLedgerRpcService dLedgerRpcSe t.setName("DLedgerServer-ScheduledExecutor"); return t; }); - this.fsmCaller = Optional.empty(); + this.fsmCaller = new StateMachineCaller(this.dLedgerStore, new NoOpStatemachine(), this.dLedgerEntryPusher); } /** @@ -141,10 +161,15 @@ public DLedgerServer(DLedgerConfig dLedgerConfig, DLedgerRpcService dLedgerRpcSe public synchronized void startup() { if (!isStarted) { this.dLedgerStore.startup(); - this.fsmCaller.ifPresent(x -> { - // Start state machine caller and load existing snapshots for data recovery - x.start(); - Optional.ofNullable(x.getSnapshotManager()).ifPresent(sm -> sm.loadSnapshot()); + this.fsmCaller.start(); + Optional.ofNullable(this.fsmCaller.getSnapshotManager()).ifPresent(x -> { + try { + x.loadSnapshot().get(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } }); if (RpcServiceMode.EXCLUSIVE.equals(this.rpcServiceMode)) { this.dLedgerRpcService.startup(); @@ -168,7 +193,7 @@ public synchronized void shutdown() { } this.dLedgerStore.shutdown(); executorService.shutdown(); - this.fsmCaller.ifPresent(StateMachineCaller::shutdown); + this.fsmCaller.shutdown(); isStarted = false; LOGGER.info("server shutdown"); } @@ -194,7 +219,7 @@ public synchronized void registerStateMachine(final StateMachine fsm) { if (this.dLedgerConfig.isEnableSnapshot()) { fsmCaller.registerSnapshotManager(new SnapshotManager(this)); } - this.fsmCaller = Optional.of(fsmCaller); + this.fsmCaller = fsmCaller; // Register state machine caller to entry pusher this.dLedgerEntryPusher.registerStateMachine(this.fsmCaller); if (dLedgerStore instanceof DLedgerMmapFileStore) { @@ -209,7 +234,7 @@ public synchronized void registerUserDefineProcessors(List } public StateMachine getStateMachine() { - return this.fsmCaller.map(StateMachineCaller::getStateMachine).orElse(null); + return this.fsmCaller.getStateMachine(); } @Override @@ -322,7 +347,7 @@ public AppendFuture appendAsLeader(List bodies) thr final AppendFuture finalFuture = future; Closure closure = new Closure() { @Override - void done(Status status) { + public void done(Status status) { AppendEntryResponse response = new AppendEntryResponse(); response.setGroup(DLedgerServer.this.memberState.getGroup()); response.setTerm(DLedgerServer.this.memberState.currTerm()); @@ -446,7 +471,22 @@ public CompletableFuture handlePush(PushEntryRequest request) response.setLeaderId(memberState.getLeaderId()); return CompletableFuture.completedFuture(response); } + } + @Override + public CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) throws Exception { + try { + PreConditions.check(memberState.getSelfId().equals(request.getRemoteId()), DLedgerResponseCode.UNKNOWN_MEMBER, "%s != %s", request.getRemoteId(), memberState.getSelfId()); + PreConditions.check(memberState.getGroup().equals(request.getGroup()), DLedgerResponseCode.UNKNOWN_GROUP, "%s != %s", request.getGroup(), memberState.getGroup()); + return dLedgerEntryPusher.handleInstallSnapshot(request); + } catch (DLedgerException e) { + LOGGER.error("[{}][HandleInstallSnapshot] failed", memberState.getSelfId(), e); + InstallSnapshotResponse response = new InstallSnapshotResponse(); + response.copyBaseInfo(request); + response.setCode(e.getCode().getCode()); + response.setLeaderId(memberState.getLeaderId()); + return CompletableFuture.completedFuture(response); + } } @Override @@ -637,7 +677,7 @@ public NettyRemotingClient getRemotingClient() { } public StateMachineCaller getFsmCaller() { - return fsmCaller.orElseThrow(NullPointerException::new); + return fsmCaller; } public boolean isLeader() { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/MemberState.java b/dledger/src/main/java/io/openmessaging/storage/dledger/MemberState.java index 59a698b2..3715732b 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/MemberState.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/MemberState.java @@ -37,22 +37,37 @@ public class MemberState { public static final String TERM_PERSIST_FILE = "currterm"; public static final String TERM_PERSIST_KEY_TERM = "currTerm"; public static final String TERM_PERSIST_KEY_VOTE_FOR = "voteLeader"; + public static Logger logger = LoggerFactory.getLogger(MemberState.class); - public final DLedgerConfig dLedgerConfig; + private final ReentrantLock defaultLock = new ReentrantLock(); + + // basic cluster info + public final DLedgerConfig dLedgerConfig; private final String group; private final String selfId; private final String peers; + private final Map peerMap = new HashMap<>(); + private final Map peersLiveTable = new ConcurrentHashMap<>(); + + // volatile states for all servers private volatile Role role = CANDIDATE; private volatile String leaderId; + private volatile long committedIndex = -1; + private volatile long appliedIndex = -1; + + private volatile long appliedTerm = -1; + + // persistent states for all servers private volatile long currTerm = 0; private volatile String currVoteFor; + private volatile long ledgerEndIndex = -1; private volatile long ledgerEndTerm = -1; private long knownMaxTermInGroup = -1; - private final Map peerMap = new HashMap<>(); - private final Map peersLiveTable = new ConcurrentHashMap<>(); + + // state for leadership transfer private volatile String transferee; private volatile long termToTakeLeadership = -1; @@ -66,10 +81,10 @@ public MemberState(DLedgerConfig config) { peerMap.put(peerSelfId, peerAddress); } this.dLedgerConfig = config; - loadTerm(); + loadPersistentState(); } - private void loadTerm() { + private void loadPersistentState() { try { String data = IOUtils.file2String(dLedgerConfig.getDefaultPath() + File.separator + TERM_PERSIST_FILE); Properties properties = IOUtils.string2Properties(data); @@ -85,12 +100,13 @@ private void loadTerm() { currVoteFor = null; } } + logger.info("[MemberState] load persistent state ok. term={} voteFor={}", currTerm, currVoteFor); } catch (Throwable t) { logger.error("Load last term failed", t); } } - private void persistTerm() { + private void persistState() { try { Properties properties = new Properties(); properties.put(TERM_PERSIST_KEY_TERM, currTerm); @@ -112,7 +128,7 @@ public String currVoteFor() { public synchronized void setCurrVoteFor(String currVoteFor) { this.currVoteFor = currVoteFor; - persistTerm(); + persistState(); } public synchronized long nextTerm() { @@ -123,7 +139,7 @@ public synchronized long nextTerm() { ++currTerm; } currVoteFor = null; - persistTerm(); + persistState(); return currTerm; } @@ -253,6 +269,53 @@ public long getLedgerEndTerm() { return ledgerEndTerm; } + public long getCommittedIndex() { + return committedIndex; + } + + /** + * update committedIndex + * @param term term of the entry which is committed. Very importantly, we can only commit the entry with current term, which also means that + * we can't commit the entry with old term. + * @param committedIndex the index of the entry which is committed. + */ + public boolean leaderUpdateCommittedIndex(long term, long committedIndex) { + if (!this.isLeader()) return false; + // prevent back to an old state + if (term != this.currTerm && committedIndex <= this.committedIndex) { + return false; + } + logger.debug("[MemberState] leader update committed index from {} to {}", this.committedIndex, committedIndex); + this.committedIndex = committedIndex; + return true; + } + + public boolean followerUpdateCommittedIndex(long committedIndex) { + if (!this.isFollower()) return false; + if (committedIndex <= this.committedIndex) { + return false; + } + logger.debug("[MemberState] follower update committed index from {} to {}", this.committedIndex, committedIndex); + this.committedIndex = committedIndex; + return true; + } + + public long getAppliedIndex() { + return appliedIndex; + } + + public long getAppliedTerm() { + return appliedTerm; + } + + public void updateAppliedIndexAndTerm(long applyIndex, long appliedTerm) { + if (appliedTerm < this.appliedTerm || applyIndex <= this.appliedIndex) { + return; + } + this.appliedIndex = applyIndex; + this.appliedTerm = appliedTerm; + } + public enum Role { UNKNOWN, CANDIDATE, diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/client/DLedgerClient.java b/dledger/src/main/java/io/openmessaging/storage/dledger/client/DLedgerClient.java index 0ff338f1..5deeb1cf 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/client/DLedgerClient.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/client/DLedgerClient.java @@ -16,7 +16,7 @@ package io.openmessaging.storage.dledger.client; -import io.openmessaging.storage.dledger.ShutdownAbleThread; +import io.openmessaging.storage.dledger.common.ShutdownAbleThread; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; @@ -143,11 +143,23 @@ public V invokeUserD return response; } request.setGroup(group); - request.setRemoteId(leaderId == null ? this.peerMap.keySet().iterator().next() : leaderId); - return dLedgerClientRpcService.invokeUserDefineRequest(request, aClass).get(); + request.setRemoteId(leaderId); + V resp = dLedgerClientRpcService.invokeUserDefineRequest(request, aClass).get(); + if (resp.getCode() == DLedgerResponseCode.NOT_LEADER.getCode()) { + waitOnUpdatingMetadata(1500, false); + if (onlyForLeader && leaderId == null) { + V response = aClass.newInstance(); + response.setCode(DLedgerResponseCode.METADATA_ERROR.getCode()); + return response; + } + request.setGroup(group); + request.setRemoteId(leaderId); + resp = dLedgerClientRpcService.invokeUserDefineRequest(request, aClass).get(); + } + return resp; } catch (Exception e) { needFreshMetadata(); - LOGGER.error("invoke user define request error", e); + LOGGER.error("invoke user define request error, request: {}", request, e); try { V response = aClass.newInstance(); response.code(DLedgerResponseCode.INTERNAL_ERROR.getCode()); diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/AppendFuture.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/AppendFuture.java similarity index 91% rename from dledger/src/main/java/io/openmessaging/storage/dledger/AppendFuture.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/AppendFuture.java index 98c4e854..ef8a0765 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/AppendFuture.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/AppendFuture.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; public class AppendFuture extends TimeoutFuture { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/BatchAppendFuture.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/BatchAppendFuture.java similarity index 92% rename from dledger/src/main/java/io/openmessaging/storage/dledger/BatchAppendFuture.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/BatchAppendFuture.java index 36d68f5f..6380e9b0 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/BatchAppendFuture.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/BatchAppendFuture.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; public class BatchAppendFuture extends AppendFuture { private long[] positions; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/Closure.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/Closure.java similarity index 85% rename from dledger/src/main/java/io/openmessaging/storage/dledger/Closure.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/Closure.java index 8491b650..cc27b249 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/Closure.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/Closure.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; public abstract class Closure { @@ -34,5 +34,5 @@ public boolean isTimeOut() { return System.currentTimeMillis() - createTime >= timeoutMs; } - abstract void done(Status status); + public abstract void done(Status status); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/NamedThreadFactory.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/NamedThreadFactory.java similarity index 95% rename from dledger/src/main/java/io/openmessaging/storage/dledger/NamedThreadFactory.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/NamedThreadFactory.java index 36ea004a..a11bbc9e 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/NamedThreadFactory.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/NamedThreadFactory.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/ReadClosure.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadClosure.java similarity index 86% rename from dledger/src/main/java/io/openmessaging/storage/dledger/ReadClosure.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadClosure.java index bba4d2ab..8c0e6cb8 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/ReadClosure.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadClosure.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; public abstract class ReadClosure extends Closure { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/ReadMode.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadMode.java similarity index 86% rename from dledger/src/main/java/io/openmessaging/storage/dledger/ReadMode.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadMode.java index 793ba4d4..3893ca40 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/ReadMode.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadMode.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; public enum ReadMode { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/ShutdownAbleThread.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ShutdownAbleThread.java similarity index 96% rename from dledger/src/main/java/io/openmessaging/storage/dledger/ShutdownAbleThread.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/ShutdownAbleThread.java index 105b2a18..4c93b4bd 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/ShutdownAbleThread.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/ShutdownAbleThread.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; import io.openmessaging.storage.dledger.utils.ResettableCountDownLatch; import java.util.concurrent.CountDownLatch; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/Status.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/Status.java similarity index 91% rename from dledger/src/main/java/io/openmessaging/storage/dledger/Status.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/Status.java index 19bed442..44ef11d7 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/Status.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/Status.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/TimeoutFuture.java b/dledger/src/main/java/io/openmessaging/storage/dledger/common/TimeoutFuture.java similarity index 92% rename from dledger/src/main/java/io/openmessaging/storage/dledger/TimeoutFuture.java rename to dledger/src/main/java/io/openmessaging/storage/dledger/common/TimeoutFuture.java index e2812b62..7122a545 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/TimeoutFuture.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/common/TimeoutFuture.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,7 +14,7 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; import java.util.concurrent.CompletableFuture; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerEntryCoder.java b/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerEntryCoder.java index 13a79ef0..b94e06d8 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerEntryCoder.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerEntryCoder.java @@ -47,6 +47,16 @@ public static void encodeIndex(long pos, int size, int magic, long index, long t byteBuffer.flip(); } + public static DLedgerIndexEntry decodeIndex(ByteBuffer byteBuffer) { + DLedgerIndexEntry indexEntry = new DLedgerIndexEntry(); + indexEntry.setMagic(byteBuffer.getInt()); + indexEntry.setPosition(byteBuffer.getLong()); + indexEntry.setSize(byteBuffer.getInt()); + indexEntry.setIndex(byteBuffer.getLong()); + indexEntry.setTerm(byteBuffer.getLong()); + return indexEntry; + } + public static DLedgerEntry decode(ByteBuffer byteBuffer) { return decode(byteBuffer, true); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerIndexEntry.java b/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerIndexEntry.java new file mode 100644 index 00000000..d9c4cef0 --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerIndexEntry.java @@ -0,0 +1,70 @@ +/* + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.entry; + +public class DLedgerIndexEntry { + + private int magic; + + private long position; + + private int size; + + private long index; + + private long term; + + public int getMagic() { + return magic; + } + + public void setMagic(int magic) { + this.magic = magic; + } + + public long getPosition() { + return position; + } + + public void setPosition(long position) { + this.position = position; + } + + public int getSize() { + return size; + } + + public void setSize(int size) { + this.size = size; + } + + public long getIndex() { + return index; + } + + public void setIndex(long index) { + this.index = index; + } + + public long getTerm() { + return term; + } + + public void setTerm(long term) { + this.term = term; + } +} diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerRequestCode.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerRequestCode.java index 75234e6d..5260af97 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerRequestCode.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerRequestCode.java @@ -29,8 +29,11 @@ public enum DLedgerRequestCode { PULL(51003, ""), PUSH(51004, ""), LEADERSHIP_TRANSFER(51005, ""), + + INSTALL_SNAPSHOT(51006, ""), USER_DEFINE_REQUEST(59999, ""); + private static Map codeMap = new HashMap<>(); static { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerResponseCode.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerResponseCode.java index ed998454..62bc5d52 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerResponseCode.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerResponseCode.java @@ -49,6 +49,7 @@ public enum DLedgerResponseCode { INDEX_LESS_THAN_LOCAL_BEGIN(419, ""), REQUEST_WITH_EMPTY_BODYS(420, ""), PUSH_REQUEST_IS_FULL(421,""), + NEWER_INSTALL_SNAPSHOT_REQUEST_EXIST(422,""), INTERNAL_ERROR(500, ""), TERM_CHANGED(501, ""), WAIT_QUORUM_ACK_TIMEOUT(502, ""), @@ -56,7 +57,8 @@ public enum DLedgerResponseCode { ILLEGAL_MEMBER_STATE(504, ""), LEADER_NOT_READY(505, ""), LEADER_TRANSFERRING(506, ""), - LOAD_SNAPSHOT_ERROR(507, ""); + LOAD_SNAPSHOT_ERROR(507, ""), + INSTALL_SNAPSHOT_ERROR(508, ""); private static Map codeMap = new HashMap<>(); diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotRequest.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotRequest.java new file mode 100644 index 00000000..f078a21a --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotRequest.java @@ -0,0 +1,67 @@ +/** + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.protocol; + +public class InstallSnapshotRequest extends RequestOrResponse { + + private long lastIncludedIndex; + + private long lastIncludedTerm; + + private byte[] data; + + public InstallSnapshotRequest() { + } + + public InstallSnapshotRequest(long lastIncludedIndex, long lastIncludedTerm, byte[] data) { + this.lastIncludedIndex = lastIncludedIndex; + this.lastIncludedTerm = lastIncludedTerm; + this.data = data; + } + + public long getLastIncludedIndex() { + return lastIncludedIndex; + } + + public void setLastIncludedIndex(long lastIncludedIndex) { + this.lastIncludedIndex = lastIncludedIndex; + } + + public long getLastIncludedTerm() { + return lastIncludedTerm; + } + + public void setLastIncludedTerm(long lastIncludedTerm) { + this.lastIncludedTerm = lastIncludedTerm; + } + + public byte[] getData() { + return data; + } + + public void setData(byte[] data) { + this.data = data; + } + + @Override + public String toString() { + return "InstallSnapshotRequest{" + + "lastIncludedIndex=" + lastIncludedIndex + + ", lastIncludedTerm=" + lastIncludedTerm + + '}'; + } +} diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotResponse.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotResponse.java new file mode 100644 index 00000000..ac911ffc --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotResponse.java @@ -0,0 +1,29 @@ +/** + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.protocol; + +public class InstallSnapshotResponse extends RequestOrResponse { + + public InstallSnapshotResponse() { + } + + @Override + public RequestOrResponse code(int code) { + this.code = code; + return this; + } +} diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryRequest.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryRequest.java index 75e12f5f..d9a4d02c 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryRequest.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryRequest.java @@ -22,22 +22,14 @@ import java.util.List; public class PushEntryRequest extends RequestOrResponse { + private long preLogIndex = -1; + private long preLogTerm = -1; private long commitIndex = -1; private Type type = Type.APPEND; - private DLedgerEntry entry; - //for batch append push - private List batchEntry = new ArrayList<>(); + private List entries = new ArrayList<>(); private int totalSize; - public DLedgerEntry getEntry() { - return entry; - } - - public void setEntry(DLedgerEntry entry) { - this.entry = entry; - } - public Type getType() { return type; } @@ -54,66 +46,76 @@ public void setCommitIndex(long commitIndex) { this.commitIndex = commitIndex; } + public long getPreLogIndex() { + return preLogIndex; + } + + public void setPreLogIndex(long preLogIndex) { + this.preLogIndex = preLogIndex; + } + + public long getPreLogTerm() { + return preLogTerm; + } + + public void setPreLogTerm(long preLogTerm) { + this.preLogTerm = preLogTerm; + } + public void addEntry(DLedgerEntry entry) { - if (!batchEntry.isEmpty()) { - PreConditions.check(batchEntry.get(0).getIndex() + batchEntry.size() == entry.getIndex(), - DLedgerResponseCode.UNKNOWN, "batch push wrong order"); + if (!entries.isEmpty()) { + PreConditions.check(entries.get(entries.size() - 1).getIndex() + 1 == entry.getIndex(), DLedgerResponseCode.UNKNOWN, "batch push in wrong order"); } - batchEntry.add(entry); + entries.add(entry); totalSize += entry.getSize(); } public long getFirstEntryIndex() { - if (!batchEntry.isEmpty()) { - return batchEntry.get(0).getIndex(); - } else if (entry != null) { - return entry.getIndex(); + if (!entries.isEmpty()) { + return entries.get(0).getIndex(); } else { return -1; } } public long getLastEntryIndex() { - if (!batchEntry.isEmpty()) { - return batchEntry.get(batchEntry.size() - 1).getIndex(); - } else if (entry != null) { - return entry.getIndex(); + if (!entries.isEmpty()) { + return entries.get(entries.size() - 1).getIndex(); } else { return -1; } } - public int getCount() { - if (!batchEntry.isEmpty()) { - return batchEntry.size(); - } else if (entry != null) { - return 1; + public long getLastEntryTerm() { + if (!entries.isEmpty()) { + return entries.get(entries.size() - 1).getTerm(); } else { - return 0; + return -1; } } + public int getCount() { + return entries.size(); + } + public long getTotalSize() { return totalSize; } - public List getBatchEntry() { - return batchEntry; + public List getEntries() { + return entries; } public void clear() { - batchEntry.clear(); + entries.clear(); totalSize = 0; } - public boolean isBatch() { - return !batchEntry.isEmpty(); - } - public enum Type { APPEND, COMMIT, COMPARE, - TRUNCATE + TRUNCATE, + INSTALL_SNAPSHOT } } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryResponse.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryResponse.java index cef3a0a4..1934b5b6 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryResponse.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryResponse.java @@ -23,6 +23,26 @@ public class PushEntryResponse extends RequestOrResponse { private long endIndex; private int count; + private long xTerm = -1; + + private long xIndex = -1; + + public long getXTerm() { + return xTerm; + } + + public void setXTerm(long xTerm) { + this.xTerm = xTerm; + } + + public long getXIndex() { + return xIndex; + } + + public void setXIndex(long xIndex) { + this.xIndex = xIndex; + } + public Long getIndex() { return index; } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerInnerProtocolHandler.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerInnerProtocolHandler.java index a927fa5f..24c40db3 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerInnerProtocolHandler.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerInnerProtocolHandler.java @@ -16,8 +16,8 @@ package io.openmessaging.storage.dledger.protocol.handler; -import io.openmessaging.storage.dledger.ReadClosure; -import io.openmessaging.storage.dledger.ReadMode; +import io.openmessaging.storage.dledger.common.ReadClosure; +import io.openmessaging.storage.dledger.common.ReadMode; public interface DLedgerInnerProtocolHandler { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRaftProtocolHandler.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRaftProtocolHandler.java index 81e1a309..ee25d40d 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRaftProtocolHandler.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRaftProtocolHandler.java @@ -18,6 +18,8 @@ import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.PullEntriesRequest; import io.openmessaging.storage.dledger.protocol.PullEntriesResponse; import io.openmessaging.storage.dledger.protocol.PushEntryRequest; @@ -37,4 +39,6 @@ public interface DLedgerRaftProtocolHandler { CompletableFuture handlePush(PushEntryRequest request) throws Exception; + CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) throws Exception; + } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerRaftProtocol.java b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerRaftProtocol.java index b37b576d..52614fe7 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerRaftProtocol.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerRaftProtocol.java @@ -18,6 +18,8 @@ import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.PullEntriesRequest; import io.openmessaging.storage.dledger.protocol.PullEntriesResponse; import io.openmessaging.storage.dledger.protocol.PushEntryRequest; @@ -37,4 +39,6 @@ public interface DLedgerRaftProtocol { CompletableFuture push(PushEntryRequest request) throws Exception; + CompletableFuture installSnapshot(InstallSnapshotRequest request) throws Exception; + } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/DownloadSnapshot.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/DownloadSnapshot.java new file mode 100644 index 00000000..06ede595 --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/DownloadSnapshot.java @@ -0,0 +1,48 @@ +/* + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.snapshot; + +public class DownloadSnapshot { + + private SnapshotMeta meta; + + private byte[] data; + + public DownloadSnapshot() { + } + + public DownloadSnapshot(SnapshotMeta meta, byte[] data) { + this.meta = meta; + this.data = data; + } + + public SnapshotMeta getMeta() { + return meta; + } + + public void setMeta(SnapshotMeta meta) { + this.meta = meta; + } + + public byte[] getData() { + return data; + } + + public void setData(byte[] data) { + this.data = data; + } +} diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotEntryResetStrategy.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotEntryResetStrategy.java new file mode 100644 index 00000000..0cbb2c4a --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotEntryResetStrategy.java @@ -0,0 +1,50 @@ +/* + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.snapshot; + +import io.openmessaging.storage.dledger.DLedgerConfig; + +public enum SnapshotEntryResetStrategy { + /** + * Delete all entries in (..., lastIncludedIndex] in sync mode + */ + RESET_ALL_SYNC, + /** + * Delete all entries in (..., lastIncludedIndex] in async mode + */ + RESET_ALL_ASYNC, + + /** + * Delete all entries in (..., lastIncludedIndex] in the configured time {@link DLedgerConfig#getResetSnapshotEntriesDelayTime()} + */ + RESET_ALL_LATER, + + /** + * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in sync mode + */ + RESET_BUT_KEEP_SOME_SYNC, + + /** + * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in async mode + */ + RESET_BUT_KEEP_SOME_ASYNC, + + /** + * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in the configured time {@link DLedgerConfig#getResetSnapshotEntriesDelayTime()} + */ + RESET_BUT_KEEP_SOME_LATER, +} \ No newline at end of file diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotManager.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotManager.java index 8be5dd0d..859da226 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotManager.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotManager.java @@ -18,13 +18,19 @@ import io.openmessaging.storage.dledger.DLedgerConfig; import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.entry.DLedgerEntry; +import io.openmessaging.storage.dledger.MemberState; +import io.openmessaging.storage.dledger.common.Closure; +import io.openmessaging.storage.dledger.common.NamedThreadFactory; +import io.openmessaging.storage.dledger.common.Status; import io.openmessaging.storage.dledger.exception.DLedgerException; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; import io.openmessaging.storage.dledger.snapshot.file.FileSnapshotStore; import io.openmessaging.storage.dledger.snapshot.hook.LoadSnapshotHook; import io.openmessaging.storage.dledger.snapshot.hook.SaveSnapshotHook; import io.openmessaging.storage.dledger.utils.IOUtils; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,6 +39,28 @@ import java.util.Objects; import java.util.concurrent.CompletableFuture; +/** + * snapshot dir tree (here is an example) + *
+ *   snapshot
+ *     - tmp  (saving snapshot now, but has not been renamed to a snapshot dir)
+ *       - snapshot_meta  (index=39, term=1)
+ *       - data  (statemachine data util index=39(included))
+ *
+ *     - snapshot_13  (means snapshot which has statemachine data util index=13(included))
+ *       - snapshot_meta  (index=13, term=1)
+ *       - data  (statemachine data util index=13(included))
+ *
+ *     - snapshot_26  (means snapshot which has statemachine data util index=26(included))
+ *       - snapshot_meta  (index=26, term=1)
+ *       - data
+ *
+ *     - install_tmp  (downloaded snapshot from leader, but has not been renamed to a snapshot dir)
+ *       - snapshot_meta
+ *       - data
+ *
+ * 
+ */ public class SnapshotManager { private static Logger logger = LoggerFactory.getLogger(SnapshotManager.class); @@ -41,17 +69,27 @@ public class SnapshotManager { public static final String SNAPSHOT_DATA_FILE = "data"; public static final String SNAPSHOT_DIR_PREFIX = "snapshot_"; public static final String SNAPSHOT_TEMP_DIR = "tmp"; + public static final String SNAPSHOT_INSTALL_TEMP_DIR = "install_tmp"; private DLedgerServer dLedgerServer; - private long lastSnapshotIndex = -1; - private long lastSnapshotTerm = -1; + + private DLedgerConfig dLedgerConfig; + private volatile long lastSnapshotIndex = -1; + private volatile long lastSnapshotTerm = -1; private final SnapshotStore snapshotStore; + + private final MemberState memberState; private volatile boolean savingSnapshot; private volatile boolean loadingSnapshot; + private final ScheduledExecutorService scheduledExecutorService; + public SnapshotManager(DLedgerServer dLedgerServer) { this.dLedgerServer = dLedgerServer; + this.dLedgerConfig = this.dLedgerServer.getDLedgerConfig(); + this.memberState = this.dLedgerServer.getMemberState(); this.snapshotStore = new FileSnapshotStore(this.dLedgerServer.getDLedgerConfig().getSnapshotStoreBaseDir()); + this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("SnapshotManager-EntriesResetService", true)); } public boolean isSavingSnapshot() { @@ -65,17 +103,15 @@ public boolean isLoadingSnapshot() { private class SaveSnapshotAfterHook implements SaveSnapshotHook { SnapshotWriter writer; - DLedgerEntry dLedgerEntry; SnapshotMeta snapshotMeta; - public SaveSnapshotAfterHook(SnapshotWriter writer, DLedgerEntry dLedgerEntry) { + public SaveSnapshotAfterHook(SnapshotWriter writer) { this.writer = writer; - this.dLedgerEntry = dLedgerEntry; } @Override public void doCallBack(SnapshotStatus status) { - saveSnapshotAfter(writer, snapshotMeta, dLedgerEntry, status); + saveSnapshotAfter(writer, snapshotMeta, status); } @Override @@ -89,10 +125,6 @@ public SnapshotWriter getSnapshotWriter() { return this.writer; } - @Override - public DLedgerEntry getSnapshotEntry() { - return this.dLedgerEntry; - } } private class LoadSnapshotAfterHook implements LoadSnapshotHook { @@ -100,13 +132,16 @@ private class LoadSnapshotAfterHook implements LoadSnapshotHook { SnapshotReader reader; SnapshotMeta snapshotMeta; - public LoadSnapshotAfterHook(SnapshotReader reader) { + Closure closure; + + public LoadSnapshotAfterHook(SnapshotReader reader, Closure closure) { this.reader = reader; + this.closure = closure; } @Override public void doCallBack(SnapshotStatus status) { - loadSnapshotAfter(reader, snapshotMeta, status); + loadSnapshotAfter(reader, snapshotMeta, status, closure); } @Override @@ -120,13 +155,13 @@ public SnapshotReader getSnapshotReader() { } } - public void saveSnapshot(DLedgerEntry dLedgerEntry) { + public void saveSnapshot() { // Check if still saving other snapshots if (this.savingSnapshot) { return; } // Check if applied index reaching the snapshot threshold - if (dLedgerEntry.getIndex() - this.lastSnapshotIndex < this.dLedgerServer.getDLedgerConfig().getSnapshotThreshold()) { + if (this.memberState.getAppliedIndex() - this.lastSnapshotIndex < this.dLedgerServer.getDLedgerConfig().getSnapshotThreshold()) { return; } // Create snapshot writer @@ -136,14 +171,14 @@ public void saveSnapshot(DLedgerEntry dLedgerEntry) { } // Start saving snapshot this.savingSnapshot = true; - SaveSnapshotAfterHook saveSnapshotAfter = new SaveSnapshotAfterHook(writer, dLedgerEntry); + SaveSnapshotAfterHook saveSnapshotAfter = new SaveSnapshotAfterHook(writer); if (!this.dLedgerServer.getFsmCaller().onSnapshotSave(saveSnapshotAfter)) { logger.error("Unable to call statemachine onSnapshotSave"); saveSnapshotAfter.doCallBack(SnapshotStatus.FAIL); } } - private void saveSnapshotAfter(SnapshotWriter writer, SnapshotMeta snapshotMeta, DLedgerEntry dLedgerEntry, SnapshotStatus status) { + private void saveSnapshotAfter(SnapshotWriter writer, SnapshotMeta snapshotMeta, SnapshotStatus status) { int res = status.getCode(); // Update snapshot meta if (res == SnapshotStatus.SUCCESS.getCode()) { @@ -160,19 +195,50 @@ private void saveSnapshotAfter(SnapshotWriter writer, SnapshotMeta snapshotMeta, this.lastSnapshotIndex = snapshotMeta.getLastIncludedIndex(); this.lastSnapshotTerm = snapshotMeta.getLastIncludedTerm(); logger.info("Snapshot {} saved successfully", snapshotMeta); - // Remove previous logs - CompletableFuture.runAsync(() -> { - truncatePrefix(dLedgerEntry); - }); + resetSnapshotAfterSave(lastSnapshotIndex, lastSnapshotTerm); } else { - logger.error("Unable to save snapshot"); + logger.error("Unable to save snapshot, res: {}", res); } this.savingSnapshot = false; } - private void truncatePrefix(DLedgerEntry entry) { + private void resetSnapshotAfterSave(long lastIncludedIndex, long lastIncludedTerm) { + switch (this.dLedgerConfig.getSnapshotEntryResetStrategy()) { + case RESET_ALL_SYNC: + truncatePrefix(lastIncludedIndex, lastIncludedTerm); + break; + case RESET_ALL_ASYNC: + CompletableFuture.runAsync(() -> { + truncatePrefix(lastIncludedIndex, lastIncludedTerm); + }); + break; + case RESET_ALL_LATER: + this.scheduledExecutorService.schedule(() -> { + truncatePrefix(lastIncludedIndex, lastIncludedTerm); + }, this.dLedgerConfig.getResetSnapshotEntriesDelayTime(), TimeUnit.MILLISECONDS); + break; + case RESET_BUT_KEEP_SOME_SYNC: + truncatePrefix(lastIncludedIndex - dLedgerConfig.getResetSnapshotEntriesButKeepLastEntriesNum(), lastIncludedTerm); + break; + case RESET_BUT_KEEP_SOME_ASYNC: + CompletableFuture.runAsync(() -> { + truncatePrefix(lastIncludedIndex - dLedgerConfig.getResetSnapshotEntriesButKeepLastEntriesNum(), lastIncludedTerm); + }); + break; + case RESET_BUT_KEEP_SOME_LATER: + this.scheduledExecutorService.schedule(() -> { + truncatePrefix(lastIncludedIndex - dLedgerConfig.getResetSnapshotEntriesButKeepLastEntriesNum(), lastIncludedTerm); + }, this.dLedgerConfig.getResetSnapshotEntriesDelayTime(), TimeUnit.MILLISECONDS); + break; + default: + logger.error("Unknown reset strategy {}", this.dLedgerConfig.getSnapshotEntryResetStrategy()); + break; + } + } + + private void truncatePrefix(long lastIncludedIndex, long lastIncludedTerm) { deleteExpiredSnapshot(); - this.dLedgerServer.getDLedgerStore().resetOffsetAfterSnapshot(entry); + this.dLedgerServer.getDLedgerStore().reset(lastIncludedIndex, lastIncludedTerm); } private void deleteExpiredSnapshot() { @@ -191,39 +257,55 @@ private void deleteExpiredSnapshot() { String deleteFilePath = config.getSnapshotStoreBaseDir() + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + minSnapshotIdx; try { IOUtils.deleteFile(new File(deleteFilePath)); + logger.info("Delete expired snapshot: {}", deleteFilePath); } catch (IOException e) { logger.error("Unable to remove expired snapshot: {}", deleteFilePath, e); } } } - public void loadSnapshot() { + public CompletableFuture loadSnapshot() { // Check if still loading snapshot if (loadingSnapshot) { - return; + return CompletableFuture.completedFuture(false); } // Create snapshot reader SnapshotReader reader = snapshotStore.createSnapshotReader(); if (reader == null) { - return; + return CompletableFuture.completedFuture(false); } + CompletableFuture future = new CompletableFuture<>(); + Closure closure = new Closure() { + @Override + public void done(Status status) { + if (status.isOk()) { + future.complete(true); + } else { + logger.error("Failed to load snapshot", status); + future.complete(false); + } + } + }; // Start loading snapshot this.loadingSnapshot = true; - LoadSnapshotAfterHook loadSnapshotAfter = new LoadSnapshotAfterHook(reader); + LoadSnapshotAfterHook loadSnapshotAfter = new LoadSnapshotAfterHook(reader, closure); if (!this.dLedgerServer.getFsmCaller().onSnapshotLoad(loadSnapshotAfter)) { this.dLedgerServer.getFsmCaller().setError(this.dLedgerServer, - new DLedgerException(DLedgerResponseCode.LOAD_SNAPSHOT_ERROR, "Unable to call statemachine onSnapshotLoad")); + new DLedgerException(DLedgerResponseCode.LOAD_SNAPSHOT_ERROR, "Unable to call statemachine onSnapshotLoad")); } + return future; } - private void loadSnapshotAfter(SnapshotReader reader, SnapshotMeta snapshotMeta, SnapshotStatus status) { + private void loadSnapshotAfter(SnapshotReader reader, SnapshotMeta snapshotMeta, SnapshotStatus status, Closure closure) { if (status.getCode() == SnapshotStatus.SUCCESS.getCode()) { this.lastSnapshotIndex = snapshotMeta.getLastIncludedIndex(); this.lastSnapshotTerm = snapshotMeta.getLastIncludedTerm(); this.loadingSnapshot = false; - this.dLedgerServer.getDLedgerStore().updateIndexAfterLoadingSnapshot(this.lastSnapshotIndex, this.lastSnapshotTerm); + this.dLedgerServer.getDLedgerStore().reset(this.lastSnapshotIndex, this.lastSnapshotTerm); + closure.done(Status.ok()); logger.info("Snapshot {} loaded successfully", snapshotMeta); } else { + closure.done(Status.error(DLedgerResponseCode.LOAD_SNAPSHOT_ERROR)); // Stop the loading process if the snapshot is expired if (status.getCode() == SnapshotStatus.EXPIRED.getCode()) { this.loadingSnapshot = false; @@ -250,7 +332,7 @@ private void loadSnapshotAfter(SnapshotReader reader, SnapshotMeta snapshotMeta, return; } this.dLedgerServer.getFsmCaller().setError(this.dLedgerServer, - new DLedgerException(DLedgerResponseCode.LOAD_SNAPSHOT_ERROR, "Fail to recover state machine")); + new DLedgerException(DLedgerResponseCode.LOAD_SNAPSHOT_ERROR, "Fail to recover state machine")); return; } // Retry loading the previous snapshots @@ -259,4 +341,56 @@ private void loadSnapshotAfter(SnapshotReader reader, SnapshotMeta snapshotMeta, loadSnapshot(); } } + + public SnapshotReader getSnapshotReaderIncludedTargetIndex(long index) { + SnapshotReader reader = this.snapshotStore.createSnapshotReader(); + try { + reader.load(); + } catch (Exception e) { + logger.error("Load snapshot reader: {} meta failed", reader.getSnapshotStorePath(), e); + return null; + } + if (reader.getSnapshotMeta().getLastIncludedIndex() < index) { + return null; + } + return reader; + } + + public boolean installSnapshot(DownloadSnapshot snapshot) { + SnapshotMeta meta = snapshot.getMeta(); + if (meta.getLastIncludedTerm() < lastSnapshotTerm || (meta.getLastIncludedTerm() == lastSnapshotTerm && meta.getLastIncludedIndex() <= lastSnapshotIndex)) { + logger.warn("Ignore installing snapshot {}, because the last applied snapshot is [term={}, index={}]", meta, lastSnapshotTerm, lastSnapshotIndex); + return false; + } + SnapshotReader lastSnpReader = snapshotStore.createSnapshotReader(); + SnapshotMeta lastSnpMeta = null; + if (lastSnpReader != null) { + try { + lastSnpMeta = lastSnpReader.load(); + } catch (Exception e) { + logger.error("Load snapshot reader: {} meta failed", lastSnpReader.getSnapshotStorePath(), e); + return false; + } + } + if (lastSnpMeta != null && + (meta.getLastIncludedTerm() < lastSnpMeta.getLastIncludedTerm() || + (meta.getLastIncludedTerm() == lastSnpMeta.getLastIncludedTerm() && meta.getLastIncludedIndex() <= lastSnpMeta.getLastIncludedIndex()))) { + logger.warn("Ignore installing snapshot {}, because the last saved snapshot is [term={}, index={}]", meta, lastSnpMeta.getLastIncludedTerm(), lastSnpMeta.getLastIncludedIndex()); + return false; + } + if (!snapshotStore.downloadSnapshot(snapshot)) { + logger.warn("Install snapshot {} failed", meta); + return false; + } + try { + return this.loadSnapshot().get(); + } catch (Exception e) { + logger.error("Install Snapshot and wait loading failed", e); + return false; + } + } + + public long getLastSnapshotIndex() { + return lastSnapshotIndex; + } } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotReader.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotReader.java index 061586c4..871b6a37 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotReader.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotReader.java @@ -28,4 +28,6 @@ public interface SnapshotReader { SnapshotMeta getSnapshotMeta(); String getSnapshotStorePath(); + + DownloadSnapshot generateDownloadSnapshot(); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStore.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStore.java index e5d1e18a..8c7e1526 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStore.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStore.java @@ -21,4 +21,6 @@ public interface SnapshotStore { SnapshotWriter createSnapshotWriter(); SnapshotReader createSnapshotReader(); + + boolean downloadSnapshot(DownloadSnapshot downloadSnapshot); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotReader.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotReader.java index 7afc6738..cd4427c5 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotReader.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotReader.java @@ -17,10 +17,12 @@ package io.openmessaging.storage.dledger.snapshot.file; import com.alibaba.fastjson.JSON; +import io.openmessaging.storage.dledger.snapshot.DownloadSnapshot; import io.openmessaging.storage.dledger.snapshot.SnapshotManager; import io.openmessaging.storage.dledger.snapshot.SnapshotMeta; import io.openmessaging.storage.dledger.snapshot.SnapshotReader; import io.openmessaging.storage.dledger.utils.IOUtils; +import java.io.FileInputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,4 +60,32 @@ public SnapshotMeta getSnapshotMeta() { public String getSnapshotStorePath() { return this.snapshotStorePath; } + + @Override + public DownloadSnapshot generateDownloadSnapshot() { + try { + load(); + } catch (Exception e) { + logger.error("load snapshot metadata error", e); + return null; + } + File dataFile = new File(this.snapshotStorePath + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); + if (!dataFile.exists()) { + logger.error("snapshot data file not exist, {}", dataFile.getAbsolutePath()); + return null; + } + DownloadSnapshot downloadSnapshot = new DownloadSnapshot(); + downloadSnapshot.setMeta(this.snapshotMeta); + byte[] data = new byte[(int) dataFile.length()]; + try { + FileInputStream inputStream = new FileInputStream(dataFile); + inputStream.read(data); + inputStream.close(); + downloadSnapshot.setData(data); + } catch (Exception e) { + logger.error("load snapshot data file error", e); + return null; + } + return downloadSnapshot; + } } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotStore.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotStore.java index 20f18182..68134d01 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotStore.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/file/FileSnapshotStore.java @@ -16,11 +16,14 @@ package io.openmessaging.storage.dledger.snapshot.file; +import io.openmessaging.storage.dledger.snapshot.DownloadSnapshot; import io.openmessaging.storage.dledger.snapshot.SnapshotManager; import io.openmessaging.storage.dledger.snapshot.SnapshotReader; +import io.openmessaging.storage.dledger.snapshot.SnapshotStatus; import io.openmessaging.storage.dledger.snapshot.SnapshotStore; import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; import io.openmessaging.storage.dledger.utils.IOUtils; +import java.io.FileOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,8 +64,12 @@ private void initStore() { @Override public SnapshotWriter createSnapshotWriter() { + return createSnapshotWriter(this.snapshotStoreBaseDir + File.separator + SnapshotManager.SNAPSHOT_TEMP_DIR); + } + + private SnapshotWriter createSnapshotWriter(String snapshotStorePath) { // Delete temp snapshot - String tmpSnapshotStorePath = this.snapshotStoreBaseDir + File.separator + SnapshotManager.SNAPSHOT_TEMP_DIR; + String tmpSnapshotStorePath = snapshotStorePath; if (new File(tmpSnapshotStorePath).exists()) { try { IOUtils.deleteFile(new File(tmpSnapshotStorePath)); @@ -90,10 +97,46 @@ public SnapshotReader createSnapshotReader() { return null; } String snapshotStorePath = this.snapshotStoreBaseDir + File.separator + - SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex; + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex; return new FileSnapshotReader(snapshotStorePath); } + @Override + public boolean downloadSnapshot(DownloadSnapshot downloadSnapshot) { + // clear temp install snapshot dir + String installTmpDir = this.snapshotStoreBaseDir + File.separator + SnapshotManager.SNAPSHOT_INSTALL_TEMP_DIR; + File installTmpDirFile = new File(installTmpDir); + if (installTmpDirFile.exists()) { + try { + IOUtils.deleteFile(installTmpDirFile); + } catch (IOException e) { + logger.error("Unable to delete temp install snapshot: {}", installTmpDir, e); + return false; + } + } + // create temp install snapshot dir + try { + IOUtils.mkDir(installTmpDirFile); + } catch (IOException e) { + logger.error("Unable to create temp install snapshot dir: {}", installTmpDir, e); + return false; + } + // write meta and data to temp install snapshot dir and then move it to snapshot store dir + try { + SnapshotWriter writer = createSnapshotWriter(installTmpDir); + writer.setSnapshotMeta(downloadSnapshot.getMeta()); + FileOutputStream fileOutputStream = new FileOutputStream(writer.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); + fileOutputStream.write(downloadSnapshot.getData()); + fileOutputStream.flush(); + fileOutputStream.close(); + writer.save(SnapshotStatus.SUCCESS); + return true; + } catch (Exception e) { + logger.error("Unable to write snapshot: {} data to install snapshot", downloadSnapshot, e); + return false; + } + } + private long getLastSnapshotIdx() { File[] snapshotFiles = new File(this.snapshotStoreBaseDir).listFiles(); long lastSnapshotIdx = -1; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SaveSnapshotHook.java b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SaveSnapshotHook.java index 337533ca..1afc5dab 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SaveSnapshotHook.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SaveSnapshotHook.java @@ -16,7 +16,6 @@ package io.openmessaging.storage.dledger.snapshot.hook; -import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.snapshot.SnapshotMeta; import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; @@ -25,6 +24,4 @@ public interface SaveSnapshotHook extends SnapshotHook { void registerSnapshotMeta(final SnapshotMeta meta); SnapshotWriter getSnapshotWriter(); - - DLedgerEntry getSnapshotEntry(); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/CommittedEntryIterator.java b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/CommittedEntryIterator.java index f1d40e0b..af927d31 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/CommittedEntryIterator.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/CommittedEntryIterator.java @@ -19,7 +19,6 @@ import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.store.DLedgerStore; import java.util.Iterator; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** @@ -31,16 +30,13 @@ public class CommittedEntryIterator implements Iterator { private final DLedgerStore dLedgerStore; private final long committedIndex; private final long firstApplyingIndex; - private final AtomicLong applyingIndex; private long currentIndex; private int completeAckNums = 0; - public CommittedEntryIterator(final DLedgerStore dLedgerStore, final long committedIndex, - final AtomicLong applyingIndex, final long lastAppliedIndex, + public CommittedEntryIterator(final DLedgerStore dLedgerStore, final long committedIndex, final long lastAppliedIndex, final Function completeEntryCallback) { this.dLedgerStore = dLedgerStore; this.committedIndex = committedIndex; - this.applyingIndex = applyingIndex; this.firstApplyingIndex = lastAppliedIndex + 1; this.currentIndex = lastAppliedIndex; this.completeEntryCallback = completeEntryCallback; @@ -59,7 +55,6 @@ public DLedgerEntry next() { ++this.currentIndex; if (this.currentIndex <= this.committedIndex) { final DLedgerEntry dLedgerEntry = this.dLedgerStore.get(this.currentIndex); - this.applyingIndex.set(this.currentIndex); return dLedgerEntry; } return null; diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/NoOpStatemachine.java b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/NoOpStatemachine.java new file mode 100644 index 00000000..09fc6b09 --- /dev/null +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/NoOpStatemachine.java @@ -0,0 +1,55 @@ +/* + * Copyright 2017-2022 The DLedger Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.openmessaging.storage.dledger.statemachine; + +import io.openmessaging.storage.dledger.exception.DLedgerException; +import io.openmessaging.storage.dledger.snapshot.SnapshotReader; +import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; + +public class NoOpStatemachine implements StateMachine { + @Override + public void onApply(CommittedEntryIterator iter) { + while (iter.hasNext()) { + iter.next(); + } + } + + @Override + public boolean onSnapshotSave(SnapshotWriter writer) { + return false; + } + + @Override + public boolean onSnapshotLoad(SnapshotReader reader) { + return false; + } + + @Override + public void onShutdown() { + + } + + @Override + public void onError(DLedgerException error) { + + } + + @Override + public String getBindDLedgerId() { + return null; + } +} diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/StateMachineCaller.java b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/StateMachineCaller.java index 79496689..3f66f2dc 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/StateMachineCaller.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/StateMachineCaller.java @@ -18,7 +18,8 @@ import io.openmessaging.storage.dledger.DLedgerEntryPusher; import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.ShutdownAbleThread; +import io.openmessaging.storage.dledger.MemberState; +import io.openmessaging.storage.dledger.common.ShutdownAbleThread; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.exception.DLedgerException; import io.openmessaging.storage.dledger.snapshot.SnapshotManager; @@ -39,7 +40,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,9 +75,8 @@ private static class ApplyTask { private final DLedgerStore dLedgerStore; private final StateMachine statemachine; private final DLedgerEntryPusher entryPusher; - private final AtomicLong lastAppliedIndex; - private long lastAppliedTerm; - private final AtomicLong applyingIndex; + + private final MemberState memberState; private final BlockingQueue taskQueue; private final ScheduledExecutorService scheduledExecutorService = Executors .newSingleThreadScheduledExecutor(new ThreadFactory() { @@ -92,15 +91,15 @@ public Thread newThread(Runnable r) { public StateMachineCaller(final DLedgerStore dLedgerStore, final StateMachine statemachine, final DLedgerEntryPusher entryPusher) { - super(StateMachineCaller.class.getName(), logger); + super("StateMachineCaller-" + dLedgerStore.getMemberState().getSelfId(), logger); this.dLedgerStore = dLedgerStore; this.statemachine = statemachine; this.entryPusher = entryPusher; - this.lastAppliedIndex = new AtomicLong(-1); - this.applyingIndex = new AtomicLong(-1); + this.memberState = dLedgerStore.getMemberState(); this.taskQueue = new LinkedBlockingQueue<>(1024); if (entryPusher != null) { this.completeEntryCallback = entryPusher::completeResponseFuture; + entryPusher.registerStateMachine(this); } else { this.completeEntryCallback = index -> true; } @@ -116,7 +115,7 @@ public StateMachine getStateMachine() { } public boolean onCommitted(final long committedIndex) { - if (committedIndex <= this.lastAppliedIndex.get()) + if (committedIndex <= this.memberState.getAppliedIndex()) return false; final ApplyTask task = new ApplyTask(); task.type = TaskType.COMMITTED; @@ -172,37 +171,34 @@ private void doCommitted(final long committedIndex) { if (this.error != null) { return; } + final long lastAppliedIndex = this.memberState.getAppliedIndex(); + if (lastAppliedIndex >= committedIndex) { + return; + } if (this.snapshotManager.isPresent() && (this.snapshotManager.get().isLoadingSnapshot() || this.snapshotManager.get().isSavingSnapshot())) { this.scheduledExecutorService.schedule(() -> { try { onCommitted(committedIndex); - logger.info("Still loading or saving snapshot, retry the commit task later"); + logger.info("Still loading or saving snapshot, retry the commit task with index: {} later", committedIndex); } catch (Throwable e) { e.printStackTrace(); } }, RETRY_ON_COMMITTED_DELAY, TimeUnit.MILLISECONDS); return; } - final long lastAppliedIndex = this.lastAppliedIndex.get(); - if (lastAppliedIndex >= committedIndex) { - return; - } - final CommittedEntryIterator iter = new CommittedEntryIterator(this.dLedgerStore, committedIndex, this.applyingIndex, lastAppliedIndex, this.completeEntryCallback); + final CommittedEntryIterator iter = new CommittedEntryIterator(this.dLedgerStore, committedIndex, lastAppliedIndex, this.completeEntryCallback); while (iter.hasNext()) { this.statemachine.onApply(iter); } final long lastIndex = iter.getIndex(); - this.lastAppliedIndex.set(lastIndex); - final DLedgerEntry dLedgerEntry = this.dLedgerStore.get(lastIndex); - if (dLedgerEntry != null) { - this.lastAppliedTerm = dLedgerEntry.getTerm(); - } + DLedgerEntry entry = this.dLedgerStore.get(lastIndex); + this.memberState.updateAppliedIndexAndTerm(lastIndex, entry.getTerm()); // Take snapshot - snapshotManager.ifPresent(x -> x.saveSnapshot(dLedgerEntry)); + snapshotManager.ifPresent(x -> x.saveSnapshot()); // Check response timeout. if (iter.getCompleteAckNums() == 0) { if (this.entryPusher != null) { - this.entryPusher.checkResponseFuturesTimeout(this.lastAppliedIndex.get() + 1); + this.entryPusher.checkResponseFuturesTimeout(this.memberState.getAppliedIndex() + 1); } } } @@ -226,7 +222,7 @@ private void doSnapshotLoad(LoadSnapshotHook loadSnapshotAfter) { // Compare snapshot meta with the last applied index and term long snapshotIndex = snapshotMeta.getLastIncludedIndex(); long snapshotTerm = snapshotMeta.getLastIncludedTerm(); - if (lastAppliedCompareToSnapshot(snapshotIndex, snapshotTerm) > 0) { + if (snapshotIndex <= this.memberState.getAppliedIndex()) { logger.warn("The snapshot loading is expired"); loadSnapshotAfter.doCallBack(SnapshotStatus.EXPIRED); return; @@ -244,26 +240,15 @@ private void doSnapshotLoad(LoadSnapshotHook loadSnapshotAfter) { return; } // Update statemachine info - this.lastAppliedIndex.set(snapshotMeta.getLastIncludedIndex()); - this.lastAppliedTerm = snapshotMeta.getLastIncludedTerm(); + this.memberState.updateAppliedIndexAndTerm(snapshotIndex, snapshotTerm); + this.memberState.leaderUpdateCommittedIndex(snapshotTerm, snapshotIndex); loadSnapshotAfter.registerSnapshotMeta(snapshotMeta); loadSnapshotAfter.doCallBack(SnapshotStatus.SUCCESS); } - private int lastAppliedCompareToSnapshot(long snapshotIndex, long snapshotTerm) { - // 1. Compare term 2. Compare index - int res = Long.compare(this.lastAppliedTerm, snapshotTerm); - if (res == 0) { - return Long.compare(this.lastAppliedIndex.get(), snapshotIndex); - } else { - return res; - } - } - private void doSnapshotSave(SaveSnapshotHook saveSnapshotAfter) { // Build and save snapshot meta - DLedgerEntry curEntry = saveSnapshotAfter.getSnapshotEntry(); - saveSnapshotAfter.registerSnapshotMeta(new SnapshotMeta(curEntry.getIndex(), curEntry.getTerm())); + saveSnapshotAfter.registerSnapshotMeta(new SnapshotMeta(this.memberState.getAppliedIndex(), this.memberState.getAppliedTerm())); SnapshotWriter writer = saveSnapshotAfter.getSnapshotWriter(); if (writer == null) { return; @@ -293,14 +278,6 @@ public void setError(DLedgerServer server, final DLedgerException error) { } } - public Long getLastAppliedIndex() { - return this.lastAppliedIndex.get(); - } - - public long getLastAppliedTerm() { - return lastAppliedTerm; - } - public void registerSnapshotManager(SnapshotManager snapshotManager) { this.snapshotManager = Optional.of(snapshotManager); } diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerMemoryStore.java b/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerMemoryStore.java index 398ed4a4..a0082f84 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerMemoryStore.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerMemoryStore.java @@ -33,9 +33,7 @@ public class DLedgerMemoryStore extends DLedgerStore { private static final Logger LOGGER = LoggerFactory.getLogger(DLedgerMemoryStore.class); private long ledgerBeforeBeginIndex = -1; - private long ledgerBeginIndex = -1; private long ledgerEndIndex = -1; - private long committedIndex = -1; private long ledgerEndTerm; private final Map cachedEntries = new ConcurrentHashMap<>(); @@ -47,6 +45,11 @@ public DLedgerMemoryStore(DLedgerConfig dLedgerConfig, MemberState memberState) this.memberState = memberState; } + @Override + public MemberState getMemberState() { + return this.memberState; + } + @Override public DLedgerEntry appendAsLeader(DLedgerEntry entry) { PreConditions.check(memberState.isLeader(), DLedgerResponseCode.NOT_LEADER); @@ -54,7 +57,6 @@ public DLedgerEntry appendAsLeader(DLedgerEntry entry) { PreConditions.check(memberState.isLeader(), DLedgerResponseCode.NOT_LEADER); PreConditions.check(memberState.getTransferee() == null, DLedgerResponseCode.LEADER_TRANSFERRING); ledgerEndIndex++; - committedIndex++; ledgerEndTerm = memberState.currTerm(); entry.setIndex(ledgerEndIndex); entry.setTerm(memberState.currTerm()); @@ -72,6 +74,27 @@ public long truncate(DLedgerEntry entry, long leaderTerm, String leaderId) { return appendAsFollower(entry, leaderTerm, leaderId).getIndex(); } + @Override + public long truncate(long truncateIndex) { + for (long i = truncateIndex; i <= ledgerEndIndex ; i++) { + this.cachedEntries.remove(truncateIndex); + } + DLedgerEntry entry = this.cachedEntries.get(truncateIndex - 1); + if (entry == null) { + ledgerEndIndex = -1; + ledgerEndTerm = -1; + } else { + ledgerEndIndex = entry.getIndex(); + ledgerEndTerm = entry.getTerm(); + } + return ledgerEndIndex; + } + + @Override + public long reset(long beforeBeginIndex, long beforeBeginTerm) { + return 0; + } + @Override public void resetOffsetAfterSnapshot(DLedgerEntry entry) { @@ -84,6 +107,25 @@ public void updateIndexAfterLoadingSnapshot(long lastIncludedIndex, long lastInc this.ledgerEndTerm = lastIncludedTerm; } + @Override + public DLedgerEntry getFirstLogOfTargetTerm(long targetTerm, long endIndex) { + DLedgerEntry entry = null; + for (long i = endIndex; i > ledgerBeforeBeginIndex ; i--) { + DLedgerEntry currentEntry = get(i); + if (currentEntry == null) { + continue; + } + if (currentEntry.getTerm() == targetTerm) { + entry = currentEntry; + continue; + } + if (currentEntry.getTerm() < targetTerm) { + break; + } + } + return entry; + } + @Override public void startup() { @@ -106,7 +148,6 @@ public DLedgerEntry appendAsFollower(DLedgerEntry entry, long leaderTerm, String } ledgerEndTerm = entry.getTerm(); ledgerEndIndex = entry.getIndex(); - committedIndex = entry.getIndex(); cachedEntries.put(entry.getIndex(), entry); updateLedgerEndIndexAndTerm(); return entry; @@ -124,20 +165,19 @@ public long getLedgerEndIndex() { return ledgerEndIndex; } - @Deprecated @Override - public long getLedgerBeginIndex() { - return ledgerBeginIndex; + public long getLedgerBeforeBeginIndex() { + return ledgerBeforeBeginIndex; } @Override - public long getLedgerBeforeBeginIndex() { - return ledgerBeforeBeginIndex; + public long getLedgerBeforeBeginTerm() { + return 0; } @Override - public long getCommittedIndex() { - return committedIndex; + public void flush() { + } @Override diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerStore.java b/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerStore.java index 9b0b0bcb..a37749b7 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerStore.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerStore.java @@ -21,9 +21,7 @@ public abstract class DLedgerStore { - public MemberState getMemberState() { - return null; - } + public abstract MemberState getMemberState(); public abstract DLedgerEntry appendAsLeader(DLedgerEntry entry); @@ -31,38 +29,47 @@ public MemberState getMemberState() { public abstract DLedgerEntry get(Long index); - public abstract long getCommittedIndex(); - - public void updateCommittedIndex(long term, long committedIndex) { - - } - public abstract long getLedgerEndTerm(); public abstract long getLedgerEndIndex(); - public abstract long getLedgerBeginIndex(); - public abstract long getLedgerBeforeBeginIndex(); + public abstract long getLedgerBeforeBeginTerm(); + protected void updateLedgerEndIndexAndTerm() { if (getMemberState() != null) { getMemberState().updateLedgerIndexAndTerm(getLedgerEndIndex(), getLedgerEndTerm()); } } - public void flush() { - - } + public abstract void flush(); public long truncate(DLedgerEntry entry, long leaderTerm, String leaderId) { return -1; } + /** + * truncate all entries in [truncateIndex ..] + * @param truncateIndex truncate process since where + * @return after truncate, store's end index + */ + public abstract long truncate(long truncateIndex); + + /** + * reset store's first entry, clear all entries in [.. beforeBeginIndex], make beforeBeginIndex + 1 to be first entry's index + * @param beforeBeginIndex after reset process, beforeBegin entry's index + * @param beforeBeginTerm after reset process, beforeBegin entry's term + * @return after reset, store's first log index + */ + public abstract long reset(long beforeBeginIndex, long beforeBeginTerm); + public abstract void resetOffsetAfterSnapshot(DLedgerEntry entry); public abstract void updateIndexAfterLoadingSnapshot(long lastIncludedIndex, long lastIncludedTerm); + public abstract DLedgerEntry getFirstLogOfTargetTerm(long targetTerm, long endIndex); + public abstract void startup(); public abstract void shutdown(); diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/DLedgerMmapFileStore.java b/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/DLedgerMmapFileStore.java index d3969a0a..f5299347 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/DLedgerMmapFileStore.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/DLedgerMmapFileStore.java @@ -18,32 +18,26 @@ import io.openmessaging.storage.dledger.DLedgerConfig; import io.openmessaging.storage.dledger.MemberState; -import io.openmessaging.storage.dledger.ShutdownAbleThread; +import io.openmessaging.storage.dledger.common.ShutdownAbleThread; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.entry.DLedgerEntryCoder; +import io.openmessaging.storage.dledger.entry.DLedgerIndexEntry; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; import io.openmessaging.storage.dledger.store.DLedgerStore; -import io.openmessaging.storage.dledger.utils.IOUtils; import io.openmessaging.storage.dledger.utils.Pair; import io.openmessaging.storage.dledger.utils.PreConditions; import io.openmessaging.storage.dledger.utils.DLedgerUtils; -import java.io.File; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class DLedgerMmapFileStore extends DLedgerStore { - - public static final String CHECK_POINT_FILE = "checkpoint"; - public static final String END_INDEX_KEY = "endIndex"; - public static final String COMMITTED_INDEX_KEY = "committedIndex"; public static final int MAGIC_1 = 1; public static final int CURRENT_MAGIC = MAGIC_1; public static final int INDEX_UNIT_SIZE = 32; @@ -52,10 +46,10 @@ public class DLedgerMmapFileStore extends DLedgerStore { public List appendHooks = new ArrayList<>(); private volatile long ledgerBeforeBeginIndex = -1; - private long ledgerBeginIndex = -1; + + private volatile long ledgerBeforeBeginTerm = -1; + private long ledgerEndIndex = -1; - private long committedIndex = -1; - private long committedPos = -1; private long ledgerEndTerm; private final DLedgerConfig dLedgerConfig; private final MemberState memberState; @@ -106,7 +100,6 @@ public void startup() { public void shutdown() { this.dataFileList.flush(0); this.indexFileList.flush(0); - persistCheckPoint(); if (enableCleanSpaceService) { cleanSpaceService.shutdown(); } @@ -163,10 +156,14 @@ public void recover() { try { long startPos = mappedFile.getFileFromOffset(); int magic = byteBuffer.getInt(); + if (magic == MmapFileList.BLANK_MAGIC_CODE) { + LOGGER.info("Find blank magic code at the file: {}", mappedFile.getFileName()); + continue; + } int size = byteBuffer.getInt(); long entryIndex = byteBuffer.getLong(); long entryTerm = byteBuffer.getLong(); - long pos = byteBuffer.getLong(); + byteBuffer.getLong(); byteBuffer.getInt(); //channel byteBuffer.getInt(); //chain crc byteBuffer.getInt(); //body crc @@ -174,7 +171,6 @@ public void recover() { PreConditions.check(magic != MmapFileList.BLANK_MAGIC_CODE && magic >= MAGIC_1 && MAGIC_1 <= CURRENT_MAGIC, DLedgerResponseCode.DISK_ERROR, "unknown magic=%d", magic); PreConditions.check(size > DLedgerEntry.HEADER_SIZE, DLedgerResponseCode.DISK_ERROR, "Size %d should > %d", size, DLedgerEntry.HEADER_SIZE); - PreConditions.check(pos == startPos, DLedgerResponseCode.DISK_ERROR, "pos %d != %d", pos, startPos); PreConditions.check(bodySize + DLedgerEntry.BODY_OFFSET == size, DLedgerResponseCode.DISK_ERROR, "size %d != %d + %d", size, bodySize, DLedgerEntry.BODY_OFFSET); SelectMmapBufferResult indexSbr = indexFileList.getData(entryIndex * INDEX_UNIT_SIZE); @@ -210,35 +206,37 @@ public void recover() { int relativePos = byteBuffer.position(); long absolutePos = mappedFile.getFileFromOffset() + relativePos; int magic = byteBuffer.getInt(); + int size = byteBuffer.getInt(); if (magic == MmapFileList.BLANK_MAGIC_CODE) { - processOffset = mappedFile.getFileFromOffset() + mappedFile.getFileSize(); - index++; - if (index >= mappedFiles.size()) { - LOGGER.info("Recover data file over, the last file {}", mappedFile.getFileName()); - break; - } else { + processOffset += size; + if (relativePos + size == mappedFile.getFileSize()) { + // next file + index++; + if (index >= mappedFiles.size()) { + LOGGER.info("Recover data file over, the last file {}", mappedFile.getFileName()); + break; + } mappedFile = mappedFiles.get(index); byteBuffer = mappedFile.sliceByteBuffer(); processOffset = mappedFile.getFileFromOffset(); LOGGER.info("Trying to recover data file {}", mappedFile.getFileName()); continue; } + byteBuffer.position(relativePos + size); + continue; } - - int size = byteBuffer.getInt(); if (size == 0) { LOGGER.info("Recover data file to the end of {} ", mappedFile.getFileName()); break; } long entryIndex = byteBuffer.getLong(); long entryTerm = byteBuffer.getLong(); - long pos = byteBuffer.getLong(); - byteBuffer.getInt(); //channel - byteBuffer.getInt(); //chain crc - byteBuffer.getInt(); //body crc + byteBuffer.getLong(); // position + byteBuffer.getInt(); // channel + byteBuffer.getInt(); // chain crc + byteBuffer.getInt(); // body crc int bodySize = byteBuffer.getInt(); - PreConditions.check(pos == absolutePos, DLedgerResponseCode.DISK_ERROR, "pos %d != %d", pos, absolutePos); PreConditions.check(bodySize + DLedgerEntry.BODY_OFFSET == size, DLedgerResponseCode.DISK_ERROR, "size %d != %d + %d", size, bodySize, DLedgerEntry.BODY_OFFSET); byteBuffer.position(relativePos + size); @@ -313,46 +311,24 @@ public void recover() { updateLedgerEndIndexAndTerm(); PreConditions.check(dataFileList.checkSelf(), DLedgerResponseCode.DISK_ERROR, "check data file order failed after recovery"); PreConditions.check(indexFileList.checkSelf(), DLedgerResponseCode.DISK_ERROR, "check index file order failed after recovery"); - //Load the committed index from checkpoint - Properties properties = loadCheckPoint(); - if (properties == null || !properties.containsKey(COMMITTED_INDEX_KEY)) { - return; - } - String committedIndexStr = String.valueOf(properties.get(COMMITTED_INDEX_KEY)).trim(); - if (committedIndexStr.length() <= 0) { - return; - } - LOGGER.info("Recover to get committed index={} from checkpoint", committedIndexStr); - updateCommittedIndex(memberState.currTerm(), Long.parseLong(committedIndexStr)); } - @Deprecated - private void reviseLedgerBeginIndex() { - //get ledger begin index - MmapFile firstFile = dataFileList.getFirstMappedFile(); - SelectMmapBufferResult sbr = firstFile.selectMappedBuffer(0); - try { - ByteBuffer tmpBuffer = sbr.getByteBuffer(); - tmpBuffer.position(firstFile.getStartPosition()); - tmpBuffer.getInt(); //magic - tmpBuffer.getInt(); //size - ledgerBeginIndex = tmpBuffer.getLong(); - indexFileList.resetOffset(ledgerBeginIndex * INDEX_UNIT_SIZE); - } finally { - SelectMmapBufferResult.release(sbr); - } - } - private void reviseLedgerBeforeBeginIndex() { // get ledger begin index + dataFileList.checkFirstFileAllBlank(); MmapFile firstFile = dataFileList.getFirstMappedFile(); SelectMmapBufferResult sbr = firstFile.selectMappedBuffer(0); try { ByteBuffer tmpBuffer = sbr.getByteBuffer(); tmpBuffer.position(firstFile.getStartPosition()); - tmpBuffer.getInt(); //magic + int magic = tmpBuffer.getInt();//magic int size = tmpBuffer.getInt();//size + if (magic == MmapFileList.BLANK_MAGIC_CODE) { + tmpBuffer.position(firstFile.getStartPosition() + size); + tmpBuffer.getInt(); + size = tmpBuffer.getInt(); + } if (size == 0) { // means that now empty entry return; @@ -364,6 +340,7 @@ private void reviseLedgerBeforeBeginIndex() { } finally { SelectMmapBufferResult.release(sbr); } + } @Override @@ -400,9 +377,6 @@ public DLedgerEntry appendAsLeader(DLedgerEntry entry) { } ledgerEndIndex++; ledgerEndTerm = memberState.currTerm(); -// if (ledgerBeginIndex == -1) { -// ledgerBeginIndex = ledgerEndIndex; -// } updateLedgerEndIndexAndTerm(); return entry; } @@ -459,6 +433,89 @@ public long truncate(DLedgerEntry entry, long leaderTerm, String leaderId) { } } + @Override + public long truncate(long truncateIndex) { + if (truncateIndex > this.ledgerEndIndex) { + return this.ledgerEndIndex; + } + DLedgerEntry firstTruncateEntry = this.get(truncateIndex); + long truncateStartPos = firstTruncateEntry.getPos(); + synchronized (this.memberState) { + if (truncateIndex > this.ledgerEndIndex) { + return this.ledgerEndIndex; + } + // truncate data file + dataFileList.truncateOffset(truncateStartPos); + if (dataFileList.getMaxWrotePosition() != truncateStartPos) { + LOGGER.warn("[TRUNCATE] truncate for data file error, try to truncate pos: {}, but after truncate, max wrote pos: {}, now try to rebuild", truncateStartPos, dataFileList.getMaxWrotePosition()); + PreConditions.check(dataFileList.rebuildWithPos(truncateStartPos), DLedgerResponseCode.DISK_ERROR, "rebuild data file truncatePos=%d", truncateStartPos); + } + reviseDataFileListFlushedWhere(truncateStartPos); + + // truncate index file + long truncateIndexFilePos = truncateIndex * INDEX_UNIT_SIZE; + indexFileList.truncateOffset(truncateIndexFilePos); + if (indexFileList.getMaxWrotePosition() != truncateIndexFilePos) { + LOGGER.warn("[TRUNCATE] truncate for index file error, try to truncate pos: {}, but after truncate, max wrote pos: {}, now try to rebuild", truncateIndexFilePos, indexFileList.getMaxWrotePosition()); + PreConditions.check(dataFileList.rebuildWithPos(truncateStartPos), DLedgerResponseCode.DISK_ERROR, "rebuild index file truncatePos=%d", truncateIndexFilePos); + } + reviseIndexFileListFlushedWhere(truncateIndexFilePos); + + // update store end index and its term + if (truncateIndex == 0) { + // now clear all entries + ledgerEndTerm = -1; + ledgerEndIndex = -1; + } else { + SelectMmapBufferResult endIndexBuf = indexFileList.getData((truncateIndex - 1) * INDEX_UNIT_SIZE, INDEX_UNIT_SIZE); + ByteBuffer buffer = endIndexBuf.getByteBuffer(); + DLedgerIndexEntry indexEntry = DLedgerEntryCoder.decodeIndex(buffer); + ledgerEndTerm = indexEntry.getTerm(); + ledgerEndIndex = indexEntry.getIndex(); + } + } + LOGGER.info("[TRUNCATE] truncateIndex: {}, after truncate, ledgerEndIndex: {} ledgerEndTerm: {}", truncateIndex, ledgerEndIndex, ledgerEndTerm); + return ledgerEndIndex; + } + + @Override + public long reset(long beforeBeginIndex, long beforeBeginTerm) { + // clear all entries in [.., beforeBeginIndex] + if (beforeBeginIndex <= this.ledgerBeforeBeginIndex) { + return this.ledgerBeforeBeginIndex + 1; + } + synchronized (this.memberState) { + if (beforeBeginIndex <= this.ledgerBeforeBeginIndex) { + return this.ledgerBeforeBeginIndex + 1; + } + if (beforeBeginIndex >= this.ledgerEndIndex) { + // after reset, we should have empty entries + SelectMmapBufferResult endIndexResult = indexFileList.getData(this.ledgerEndIndex * INDEX_UNIT_SIZE); + if (endIndexResult != null) { + DLedgerIndexEntry resetEntry = DLedgerEntryCoder.decodeIndex(endIndexResult.getByteBuffer()); + this.dataFileList.resetOffset(resetEntry.getPosition() + resetEntry.getSize()); + endIndexResult.release(); + } + this.indexFileList.rebuildWithPos((beforeBeginIndex + 1) * INDEX_UNIT_SIZE); + } else { + SelectMmapBufferResult data = indexFileList.getData((beforeBeginIndex + 1) * INDEX_UNIT_SIZE); + DLedgerIndexEntry resetEntry = DLedgerEntryCoder.decodeIndex(data.getByteBuffer()); + data.release(); + this.dataFileList.resetOffset(resetEntry.getPosition()); + this.indexFileList.resetOffset((beforeBeginIndex + 1) * INDEX_UNIT_SIZE); + } + this.ledgerBeforeBeginIndex = beforeBeginIndex; + this.ledgerBeforeBeginTerm = beforeBeginTerm; + if (beforeBeginIndex >= this.ledgerEndIndex) { + this.ledgerEndIndex = beforeBeginIndex; + this.ledgerEndTerm = beforeBeginTerm; + } + } + LOGGER.info("reset to beforeBeginIndex: {}, beforeBeginTerm: {}, now beforeBeginIndex: {}, beforeBeginTerm: {}, endIndex: {}, endTerm: {}", + beforeBeginIndex, beforeBeginTerm, ledgerBeforeBeginIndex, ledgerBeforeBeginTerm, ledgerEndIndex, ledgerEndTerm); + return ledgerBeforeBeginIndex + 1; + } + private void reviseDataFileListFlushedWhere(long truncatePos) { long offset = calculateWherePosition(this.dataFileList, truncatePos); LOGGER.info("Revise dataFileList flushedWhere from {} to {}", this.dataFileList.getFlushedWhere(), offset); @@ -475,7 +532,7 @@ private void reviseIndexFileListFlushedWhere(long truncateIndexOffset) { /** * calculate wherePosition after truncate * - * @param mappedFileList this.dataFileList or this.indexFileList + * @param mappedFileList this.dataFileList or this.indexFileList * @param continuedBeginOffset new begining of offset */ private long calculateWherePosition(final MmapFileList mappedFileList, long continuedBeginOffset) { @@ -531,58 +588,30 @@ public DLedgerEntry appendAsFollower(DLedgerEntry entry, long leaderTerm, String PreConditions.check(leaderTerm == memberState.currTerm(), DLedgerResponseCode.INCONSISTENT_TERM, null); PreConditions.check(leaderId.equals(memberState.getLeaderId()), DLedgerResponseCode.INCONSISTENT_LEADER, null); long dataPos = dataFileList.append(dataBuffer.array(), 0, dataBuffer.remaining()); - PreConditions.check(dataPos == entry.getPos(), DLedgerResponseCode.DISK_ERROR, "%d != %d", dataPos, entry.getPos()); DLedgerEntryCoder.encodeIndex(dataPos, entrySize, entry.getMagic(), entry.getIndex(), entry.getTerm(), indexBuffer); long indexPos = indexFileList.append(indexBuffer.array(), 0, indexBuffer.remaining(), false); PreConditions.check(indexPos == entry.getIndex() * INDEX_UNIT_SIZE, DLedgerResponseCode.DISK_ERROR, null); ledgerEndTerm = entry.getTerm(); ledgerEndIndex = entry.getIndex(); -// if (ledgerBeginIndex == -1) { -// ledgerBeginIndex = ledgerEndIndex; -// } updateLedgerEndIndexAndTerm(); return entry; } } - void persistCheckPoint() { - try { - Properties properties = new Properties(); - properties.put(END_INDEX_KEY, getLedgerEndIndex()); - properties.put(COMMITTED_INDEX_KEY, getCommittedIndex()); - String data = IOUtils.properties2String(properties); - IOUtils.string2File(data, dLedgerConfig.getDefaultPath() + File.separator + CHECK_POINT_FILE); - } catch (Throwable t) { - LOGGER.error("Persist checkpoint failed", t); - } - } - - Properties loadCheckPoint() { - try { - String data = IOUtils.file2String(dLedgerConfig.getDefaultPath() + File.separator + CHECK_POINT_FILE); - return IOUtils.string2Properties(data); - } catch (Throwable t) { - LOGGER.error("Load checkpoint failed", t); - - } - return null; - } - @Override public long getLedgerEndIndex() { return ledgerEndIndex; } - @Deprecated @Override - public long getLedgerBeginIndex() { - return ledgerBeginIndex; + public long getLedgerBeforeBeginIndex() { + return ledgerBeforeBeginIndex; } @Override - public long getLedgerBeforeBeginIndex() { - return ledgerBeforeBeginIndex; + public long getLedgerBeforeBeginTerm() { + return ledgerBeforeBeginTerm; } @Override @@ -601,7 +630,6 @@ public DLedgerEntry get(Long index) { PreConditions.check(dataSbr != null && dataSbr.getByteBuffer() != null, DLedgerResponseCode.DISK_ERROR, "Get null data for %d", index); DLedgerEntry dLedgerEntry = DLedgerEntryCoder.decode(dataSbr.getByteBuffer()); - PreConditions.check(pos == dLedgerEntry.getPos(), DLedgerResponseCode.DISK_ERROR, "%d != %d", pos, dLedgerEntry.getPos()); return dLedgerEntry; } finally { SelectMmapBufferResult.release(indexSbr); @@ -609,6 +637,25 @@ public DLedgerEntry get(Long index) { } } + @Override + public DLedgerEntry getFirstLogOfTargetTerm(long targetTerm, long endIndex) { + DLedgerEntry entry = null; + for (long i = endIndex; i > ledgerBeforeBeginIndex; i--) { + DLedgerEntry currentEntry = get(i); + if (currentEntry == null) { + continue; + } + if (currentEntry.getTerm() == targetTerm) { + entry = currentEntry; + continue; + } + if (currentEntry.getTerm() < targetTerm) { + break; + } + } + return entry; + } + private Pair getEntryPosAndSize(Long index) { indexCheck(index); SelectMmapBufferResult indexSbr = null; @@ -631,45 +678,11 @@ private void indexCheck(Long index) { PreConditions.check(index <= ledgerEndIndex, DLedgerResponseCode.INDEX_OUT_OF_RANGE, "%d should between (%d-%d]", index, ledgerBeforeBeginIndex, ledgerEndIndex); } - @Override - public long getCommittedIndex() { - return committedIndex; - } - - @Override - public void updateCommittedIndex(long term, long newCommittedIndex) { - if (newCommittedIndex == -1 - || ledgerEndIndex == -1 - || term < memberState.currTerm() - || newCommittedIndex == this.committedIndex) { - return; - } - if (newCommittedIndex < this.committedIndex - || newCommittedIndex <= this.ledgerBeforeBeginIndex) { - LOGGER.warn("[MONITOR]Skip update committed index for new={} < old={} or new={} <= beforeBeginIndex={}", newCommittedIndex, this.committedIndex, newCommittedIndex, this.ledgerBeforeBeginIndex); - return; - } - long endIndex = ledgerEndIndex; - if (newCommittedIndex > endIndex) { - //If the node fall behind too much, the committedIndex will be larger than enIndex. - newCommittedIndex = endIndex; - } - Pair posAndSize = getEntryPosAndSize(newCommittedIndex); - PreConditions.check(posAndSize != null, DLedgerResponseCode.DISK_ERROR); - this.committedIndex = newCommittedIndex; - - this.committedPos = posAndSize.getKey() + posAndSize.getValue(); - } - @Override public long getLedgerEndTerm() { return ledgerEndTerm; } - public long getCommittedPos() { - return committedPos; - } - public void addAppendHook(AppendHook writeHook) { if (!appendHooks.contains(writeHook)) { appendHooks.add(writeHook); @@ -716,7 +729,8 @@ public FlushDataService(String name, Logger logger) { super(name, logger); } - @Override public void doWork() { + @Override + public void doWork() { try { long start = System.currentTimeMillis(); DLedgerMmapFileStore.this.dataFileList.flush(0); @@ -727,7 +741,6 @@ public FlushDataService(String name, Logger logger) { } if (DLedgerUtils.elapsed(lastCheckPointTimeMs) > dLedgerConfig.getCheckPointInterval()) { - persistCheckPoint(); lastCheckPointTimeMs = System.currentTimeMillis(); } @@ -748,7 +761,8 @@ public CleanSpaceService(String name, Logger logger) { super(name, logger); } - @Override public void doWork() { + @Override + public void doWork() { try { storeBaseRatio = DLedgerUtils.getDiskPartitionSpaceUsedPercent(dLedgerConfig.getStoreBaseDir()); dataRatio = calcDataStorePathPhysicRatio(); @@ -790,17 +804,17 @@ private boolean isTimeToDelete() { private boolean isNeedCheckExpired() { return storeBaseRatio > dLedgerConfig.getDiskSpaceRatioToCheckExpired() - || dataRatio > dLedgerConfig.getDiskSpaceRatioToCheckExpired(); + || dataRatio > dLedgerConfig.getDiskSpaceRatioToCheckExpired(); } private boolean isNeedForceClean() { return storeBaseRatio > dLedgerConfig.getDiskSpaceRatioToForceClean() - || dataRatio > dLedgerConfig.getDiskSpaceRatioToForceClean(); + || dataRatio > dLedgerConfig.getDiskSpaceRatioToForceClean(); } private boolean isNeedForbiddenWrite() { return storeBaseRatio > dLedgerConfig.getDiskFullRatio() - || dataRatio > dLedgerConfig.getDiskFullRatio(); + || dataRatio > dLedgerConfig.getDiskFullRatio(); } public double calcDataStorePathPhysicRatio() { diff --git a/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/MmapFileList.java b/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/MmapFileList.java index 46856b86..fe0bc7f8 100644 --- a/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/MmapFileList.java +++ b/dledger/src/main/java/io/openmessaging/storage/dledger/store/file/MmapFileList.java @@ -75,7 +75,7 @@ public boolean checkSelf() { /** * check if first file is full of blank (only happens in first file) */ - private void checkFirstFileAllBlank() { + public void checkFirstFileAllBlank() { MmapFile firstMappedFile = getFirstMappedFile(); if (firstMappedFile == null) return; ByteBuffer byteBuffer = firstMappedFile.sliceByteBuffer(); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndGetTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndGetTest.java index 7fc69195..f1fe55f6 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndGetTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndGetTest.java @@ -22,6 +22,8 @@ import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; +import io.openmessaging.storage.dledger.util.FileTestUtil; +import java.io.File; import java.util.ArrayList; import java.util.List; import java.util.UUID; @@ -31,6 +33,8 @@ public class AppendAndGetTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("AppendAndGetTest"); + @Test public void testSingleServerInMemory() throws Exception { String group = UUID.randomUUID().toString(); @@ -161,4 +165,9 @@ public void testThreeServerInFileWithAsyncRequests() throws Exception { Assertions.assertEquals(entry.getPos(), future.get().getPos()); } } + + @Override + protected String getBaseDir() { + return STORE_PATH; + } } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndPushTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndPushTest.java index 5f5ae38e..af91d86f 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndPushTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndPushTest.java @@ -16,11 +16,14 @@ package io.openmessaging.storage.dledger; +import io.openmessaging.storage.dledger.common.AppendFuture; +import io.openmessaging.storage.dledger.common.BatchAppendFuture; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.BatchAppendEntryRequest; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; +import io.openmessaging.storage.dledger.util.FileTestUtil; import io.openmessaging.storage.dledger.utils.DLedgerUtils; import java.util.ArrayList; import java.util.LinkedList; @@ -39,6 +42,8 @@ public class AppendAndPushTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("appendAndPushTest"); + @Test public void testPushCommittedIndex() throws Exception { String group = UUID.randomUUID().toString(); @@ -56,14 +61,14 @@ public void testPushCommittedIndex() throws Exception { futures.add(future); } Assertions.assertEquals(9, dLedgerServer0.getDLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(-1, dLedgerServer0.getDLedgerStore().getCommittedIndex()); + Assertions.assertEquals(-1, dLedgerServer0.getMemberState().getCommittedIndex()); DLedgerServer dLedgerServer1 = launchServer(group, peers, "n1", "n0", DLedgerConfig.FILE); long start = System.currentTimeMillis(); - while (DLedgerUtils.elapsed(start) < 3000 && dLedgerServer1.getDLedgerStore().getCommittedIndex() != 9) { + while (DLedgerUtils.elapsed(start) < 3000 && dLedgerServer1.getMemberState().getCommittedIndex() != 9) { DLedgerUtils.sleep(100); } - Assertions.assertEquals(9, dLedgerServer0.getDLedgerStore().getCommittedIndex()); - Assertions.assertEquals(9, dLedgerServer1.getDLedgerStore().getCommittedIndex()); + Assertions.assertEquals(9, dLedgerServer0.getMemberState().getCommittedIndex()); + Assertions.assertEquals(9, dLedgerServer1.getMemberState().getCommittedIndex()); } @Test @@ -268,4 +273,9 @@ public void testBatchAppend() throws Exception { Assertions.assertEquals(count - 1, dLedgerServer1.getDLedgerStore().getLedgerEndIndex()); Thread.sleep(1000); } + + @Override + protected String getBaseDir() { + return STORE_PATH; + } } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndReadTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndReadTest.java index c2802434..5e800c6b 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndReadTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/AppendAndReadTest.java @@ -23,6 +23,7 @@ import io.openmessaging.storage.dledger.statemachine.register.RegisterReadRequest; import io.openmessaging.storage.dledger.statemachine.register.RegisterReadResponse; import io.openmessaging.storage.dledger.statemachine.register.RegisterStateMachine; +import io.openmessaging.storage.dledger.util.FileTestUtil; import io.openmessaging.storage.dledger.utils.BytesUtil; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -32,15 +33,65 @@ public class AppendAndReadTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("AppendAndReadTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } @Test - public void testSingleServerInMemory() throws Exception { + public void testSingleServerInMemory() { String group = UUID.randomUUID().toString(); String selfId = "n0"; - String peers = "n0-localhost:11001"; - DLedgerServer dLedgerServer = launchServerWithStateMachineEnableSnapshot(group, peers, selfId, selfId, DLedgerConfig.MEMORY, - 100000, 102400, new RegisterStateMachine()); + String peers = "n0-localhost:" + nextPort(); + DLedgerServer dLedgerServer = launchServerWithStateMachineDisableSnapshot(group, peers, selfId, selfId, DLedgerConfig.MEMORY, + 102400, new RegisterStateMachine()); dLedgerServer.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer))); + appendAndRead(group, peers); + } + + @Test + public void testSingleServerInFile() { + String group = UUID.randomUUID().toString(); + String selfId = "n0"; + String peers = "n0-localhost:" + nextPort(); + DLedgerServer dLedgerServer = launchServerWithStateMachineDisableSnapshot(group, peers, selfId, selfId, DLedgerConfig.FILE, + 102400, new RegisterStateMachine()); + dLedgerServer.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer))); + + appendAndRead(group, peers); + } + + @Test + public void testThreeServerInMemory() { + String group = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", nextPort(), nextPort(), nextPort()); + DLedgerServer dLedgerServer0 = launchServerWithStateMachineDisableSnapshot(group, peers, "n0", "n1", DLedgerConfig.MEMORY, 102400, new RegisterStateMachine()); + DLedgerServer dLedgerServer1 = launchServerWithStateMachineDisableSnapshot(group, peers, "n1", "n1", DLedgerConfig.MEMORY, 102400, new RegisterStateMachine()); + DLedgerServer dLedgerServer2 = launchServerWithStateMachineDisableSnapshot(group, peers, "n2", "n1", DLedgerConfig.MEMORY, 102400, new RegisterStateMachine()); + dLedgerServer0.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer0))); + dLedgerServer1.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer1))); + dLedgerServer2.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer2))); + + appendAndRead(group, peers.split(";")[1]); + } + + @Test + public void testThreeServerInFile() { + String group = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", nextPort(), nextPort(), nextPort()); + DLedgerServer dLedgerServer0 = launchServerWithStateMachineDisableSnapshot(group, peers, "n0", "n1", DLedgerConfig.FILE, 102400, new RegisterStateMachine()); + DLedgerServer dLedgerServer1 = launchServerWithStateMachineDisableSnapshot(group, peers, "n1", "n1", DLedgerConfig.FILE, 102400, new RegisterStateMachine()); + DLedgerServer dLedgerServer2 = launchServerWithStateMachineDisableSnapshot(group, peers, "n2", "n1", DLedgerConfig.FILE, 102400, new RegisterStateMachine()); + dLedgerServer0.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer0))); + dLedgerServer1.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer1))); + dLedgerServer2.registerUserDefineProcessors(Collections.singletonList(new RegisterReadProcessor(dLedgerServer2))); + + appendAndRead(group, peers.split(";")[1]); + } + + private void appendAndRead(String group, String peers) { DLedgerClient dLedgerClient = launchClient(group, peers); for (int i = 1; i <= 10; i++) { @@ -68,6 +119,7 @@ public void testSingleServerInMemory() throws Exception { Assertions.assertEquals(i * 20, resp.getValue()); } } + private AppendEntryResponse appendKeyValue(int key, int value, DLedgerClient client) { byte[] keyBytes = BytesUtil.intToBytes(key); byte[] valueBytes = BytesUtil.intToBytes(value); @@ -79,7 +131,7 @@ private AppendEntryResponse appendKeyValue(int key, int value, DLedgerClient cli private RegisterReadResponse readKeyValue(int key, DLedgerClient client) { RegisterReadRequest registerReadRequest = new RegisterReadRequest(key); - RegisterReadResponse response = client.invokeUserDefineRequest(registerReadRequest, RegisterReadResponse.class,true); + RegisterReadResponse response = client.invokeUserDefineRequest(registerReadRequest, RegisterReadResponse.class, true); Assertions.assertNotNull(response); return response; } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/BatchPushTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/BatchPushTest.java index 040ccea9..3fa5febc 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/BatchPushTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/BatchPushTest.java @@ -17,11 +17,13 @@ package io.openmessaging.storage.dledger; import io.openmessaging.storage.dledger.client.DLedgerClient; +import io.openmessaging.storage.dledger.common.AppendFuture; import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; +import io.openmessaging.storage.dledger.util.FileTestUtil; import java.util.ArrayList; import java.util.List; import java.util.UUID; @@ -36,7 +38,14 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; -public class BatchPushTest extends ServerTestHarness{ +public class BatchPushTest extends ServerTestHarness { + + public static final String STORE_PATH = FileTestUtil.createTestDir("BatchPushTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } @Test public void testBatchPushWithOneByOneRequests() throws Exception { String group = UUID.randomUUID().toString(); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/LeaderElectorTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/LeaderElectorTest.java index 7aaddd57..6d5d2e10 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/LeaderElectorTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/LeaderElectorTest.java @@ -19,6 +19,7 @@ import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; +import io.openmessaging.storage.dledger.util.FileTestUtil; import io.openmessaging.storage.dledger.utils.DLedgerUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -30,6 +31,13 @@ public class LeaderElectorTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("LeaderElectorTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } + @Test public void testSingleServer() throws Exception { String group = UUID.randomUUID().toString(); @@ -166,7 +174,7 @@ public void testThreeServerAndShutdownLeader() throws Exception { } } start = System.currentTimeMillis(); - while (parseServers(leftServers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 3 * leaderServer.getDLedgerConfig().getHeartBeatTimeIntervalMs()) { + while (parseServers(leftServers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 5 * leaderServer.getDLedgerConfig().getHeartBeatTimeIntervalMs()) { Thread.sleep(100); } Thread.sleep(1000); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/ServerTestHarness.java b/dledger/src/test/java/io/openmessaging/storage/dledger/ServerTestHarness.java index bf6c600a..c07e611f 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/ServerTestHarness.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/ServerTestHarness.java @@ -24,12 +24,14 @@ import java.util.concurrent.atomic.AtomicInteger; public class ServerTestHarness extends ServerTestBase { - + protected String getBaseDir() { + return FileTestUtil.TEST_BASE; + } protected synchronized DLedgerServer launchServer(String group, String peers, String selfId) { DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(getBaseDir() + File.separator + group); config.group(group).selfId(selfId).peers(peers); - config.setStoreType(DLedgerConfig.MEMORY); + config.setStoreType(DLedgerConfig.FILE); DLedgerServer dLedgerServer = new DLedgerServer(config); dLedgerServer.startup(); bases.add(config.getDefaultPath()); @@ -39,9 +41,9 @@ protected synchronized DLedgerServer launchServer(String group, String peers, St protected synchronized DLedgerServer launchServer(String group, String peers, String selfId, String preferredLeaderId) { DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(getBaseDir() + File.separator + group); config.group(group).selfId(selfId).peers(peers); - config.setStoreType(DLedgerConfig.MEMORY); + config.setStoreType(DLedgerConfig.FILE); config.setPreferredLeaderId(preferredLeaderId); DLedgerServer dLedgerServer = new DLedgerServer(config); dLedgerServer.startup(); @@ -53,7 +55,7 @@ protected synchronized DLedgerServer launchServer(String group, String peers, St String storeType) { DLedgerConfig config = new DLedgerConfig(); config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(getBaseDir() + File.separator + group); config.setStoreType(storeType); config.setMappedFileSizeForEntryData(10 * 1024 * 1024); config.setEnableLeaderElector(false); @@ -92,7 +94,7 @@ protected synchronized DLedgerServer launchServerWithStateMachine(String group, StateMachine stateMachine) { DLedgerConfig config = new DLedgerConfig(); config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(getBaseDir() + File.separator + group); config.setStoreType(storeType); config.setEnableSnapshot(enableSnapshot); config.setSnapshotThreshold(snapshotThreshold); @@ -120,14 +122,14 @@ protected synchronized DLedgerServer launchServerEnableBatchPush(String group, S String leaderId, String storeType) { DLedgerConfig config = new DLedgerConfig(); config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(getBaseDir() + File.separator + group); config.setStoreType(storeType); config.setMappedFileSizeForEntryData(10 * 1024 * 1024); config.setEnableLeaderElector(false); config.setEnableDiskForceClean(false); config.setDiskSpaceRatioToForceClean(0.90f); - config.setEnableBatchPush(true); - config.setMaxBatchPushSize(300); + config.setEnableBatchAppend(true); + config.setMaxBatchAppendSize(300); DLedgerServer dLedgerServer = new DLedgerServer(config); MemberState memberState = dLedgerServer.getMemberState(); memberState.setCurrTermForTest(0); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/NamedThreadFactoryTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/common/NamedThreadFactoryTest.java similarity index 91% rename from dledger/src/test/java/io/openmessaging/storage/dledger/NamedThreadFactoryTest.java rename to dledger/src/test/java/io/openmessaging/storage/dledger/common/NamedThreadFactoryTest.java index fbd64bb4..0c20a800 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/NamedThreadFactoryTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/common/NamedThreadFactoryTest.java @@ -5,7 +5,7 @@ * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * - * https://www.apache.org/licenses/LICENSE-2.0 + * https://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -14,8 +14,9 @@ * limitations under the License. */ -package io.openmessaging.storage.dledger; +package io.openmessaging.storage.dledger.common; +import io.openmessaging.storage.dledger.common.NamedThreadFactory; import java.util.concurrent.atomic.AtomicInteger; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/HeartbeatRequestTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/HeartbeatRequestTest.java index 7c5fd076..722dd247 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/HeartbeatRequestTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/HeartbeatRequestTest.java @@ -18,6 +18,7 @@ import io.openmessaging.storage.dledger.DLedgerServer; import io.openmessaging.storage.dledger.ServerTestHarness; +import io.openmessaging.storage.dledger.util.FileTestUtil; import io.openmessaging.storage.dledger.utils.DLedgerUtils; import java.util.UUID; import org.junit.jupiter.api.Assertions; @@ -25,6 +26,12 @@ public class HeartbeatRequestTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("HeartbeatRequestTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } @Test public void testHeartbeat() throws Exception { String group = UUID.randomUUID().toString(); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/VoteRequestTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/VoteRequestTest.java index f75ba74a..595cd4e9 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/VoteRequestTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/protocol/VoteRequestTest.java @@ -18,6 +18,7 @@ import io.openmessaging.storage.dledger.DLedgerServer; import io.openmessaging.storage.dledger.ServerTestHarness; +import io.openmessaging.storage.dledger.util.FileTestUtil; import io.openmessaging.storage.dledger.utils.DLedgerUtils; import java.util.UUID; import org.junit.jupiter.api.Assertions; @@ -25,6 +26,13 @@ public class VoteRequestTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("voteRequestTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } + @Test public void testVoteNormal() throws Exception { String group = UUID.randomUUID().toString(); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotManagerTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotManagerTest.java index bbb628db..f570d12d 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotManagerTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotManagerTest.java @@ -3,6 +3,7 @@ import com.alibaba.fastjson.JSON; import io.openmessaging.storage.dledger.DLedgerConfig; import io.openmessaging.storage.dledger.DLedgerServer; +import io.openmessaging.storage.dledger.MemberState; import io.openmessaging.storage.dledger.ServerTestHarness; import io.openmessaging.storage.dledger.client.DLedgerClient; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; @@ -23,6 +24,12 @@ public class SnapshotManagerTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("SnapshotManagerTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } @Test public void testSaveAndLoadSnapshot() throws InterruptedException { @@ -32,25 +39,47 @@ public void testSaveAndLoadSnapshot() throws InterruptedException { DLedgerServer dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n1", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); DLedgerServer dLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n1", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); DLedgerServer dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n1", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); - final List serverList = new ArrayList() { - { - add(dLedgerServer0); - add(dLedgerServer1); - add(dLedgerServer2); - } - }; + final List serverList = new ArrayList(3); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer1); + serverList.add(dLedgerServer2); // Launch client DLedgerClient dLedgerClient = launchClient(group, peers.split(";")[0]); // append 99 entries, each 10 entries will trigger one snapshotting for (int i = 0; i < 99; i++) { + if (i % 10 == 0) { + Thread.sleep(200); + } AppendEntryResponse appendEntryResponse = dLedgerClient.append(new byte[512]); assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); assertEquals(i, appendEntryResponse.getIndex()); } Thread.sleep(2000); + for (DLedgerServer server : serverList) { + MockStateMachine stateMachine = (MockStateMachine) server.getStateMachine(); + assertEquals(98, server.getDLedgerStore().getLedgerEndIndex()); + assertEquals(stateMachine.getLastSnapshotIncludedIndex(), server.getDLedgerStore().getLedgerBeforeBeginIndex()); + // check statemachine + assertEquals(99, stateMachine.getTotalEntries()); + } + + // now restart, expect to load the latest snapshot and replay the entries after loaded snapshot + // Shutdown server + dLedgerServer0.shutdown(); + dLedgerServer1.shutdown(); + dLedgerServer2.shutdown(); + serverList.clear(); + // Restart server and apply snapshot + dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer1); + serverList.add(dLedgerServer2); + Thread.sleep(2000); + // State machine could only be recovered from snapshot due to the entry has been removed after saving snapshot for (DLedgerServer server : serverList) { assertEquals(98, server.getDLedgerStore().getLedgerEndIndex()); - assertEquals(89, server.getDLedgerStore().getLedgerBeforeBeginIndex()); // check statemachine final MockStateMachine fsm = (MockStateMachine) server.getStateMachine(); assertEquals(99, fsm.getTotalEntries()); @@ -63,11 +92,11 @@ public void testSaveAndLoadSnapshot() throws InterruptedException { assertEquals(99, appendEntryResponse.getIndex()); Thread.sleep(2000); for (DLedgerServer server : serverList) { + MockStateMachine stateMachine = (MockStateMachine) server.getStateMachine(); assertEquals(99, server.getDLedgerStore().getLedgerEndIndex()); - assertEquals(99, server.getDLedgerStore().getLedgerBeforeBeginIndex()); + assertEquals(stateMachine.getLastSnapshotIncludedIndex(), server.getDLedgerStore().getLedgerBeforeBeginIndex()); // check statemachine - final MockStateMachine fsm = (MockStateMachine) server.getStateMachine(); - assertEquals(100, fsm.getTotalEntries()); + assertEquals(100, stateMachine.getTotalEntries()); } Thread.sleep(100); @@ -77,21 +106,118 @@ public void testSaveAndLoadSnapshot() throws InterruptedException { dLedgerServer2.shutdown(); serverList.clear(); // Restart server and apply snapshot - DLedgerServer newDLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); - DLedgerServer newDLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); - DLedgerServer newDLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); - serverList.add(newDLedgerServer0); - serverList.add(newDLedgerServer1); - serverList.add(newDLedgerServer2); + dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer1); + serverList.add(dLedgerServer2); + Thread.sleep(2000); + // State machine could only be recovered from snapshot due to the entry has been removed after saving snapshot + for (DLedgerServer server : serverList) { + assertEquals(99, server.getDLedgerStore().getLedgerEndIndex()); + // check statemachine + final MockStateMachine fsm = (MockStateMachine) server.getStateMachine(); + assertEquals(100, fsm.getTotalEntries()); + } + } + + @Test + public void testInstallSnapshot() throws Exception { + // Launch server + String group = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", nextPort(), nextPort(), nextPort()); + DLedgerServer dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + DLedgerServer dLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + // DLedgerServer dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n1", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + final List serverList = new ArrayList(); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer1); + // Launch client + DLedgerClient dLedgerClient = launchClient(group, peers.split(";")[0]); + // append 99 entries, each 10 entries will trigger one snapshotting + for (int i = 0; i < 99; i++) { + if (i % 10 == 0) { + Thread.sleep(200); + } + AppendEntryResponse appendEntryResponse = dLedgerClient.append(new byte[512]); + assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); + assertEquals(i, appendEntryResponse.getIndex()); + } + Thread.sleep(2000); + for (DLedgerServer server : serverList) { + MockStateMachine stateMachine = (MockStateMachine) server.getStateMachine(); + assertEquals(98, server.getDLedgerStore().getLedgerEndIndex()); + assertEquals(stateMachine.getLastSnapshotIncludedIndex(), server.getDLedgerStore().getLedgerBeforeBeginIndex()); + // check statemachine + assertEquals(99, stateMachine.getTotalEntries()); + } + + // now we append an entry will trigger the snapshotting + // this time will delete entries on a scale of 90 to 99 + AppendEntryResponse appendEntryResponse = dLedgerClient.append(new byte[512]); + assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); + assertEquals(99, appendEntryResponse.getIndex()); + Thread.sleep(2000); + for (DLedgerServer server : serverList) { + MockStateMachine stateMachine = (MockStateMachine) server.getStateMachine(); + assertEquals(99, server.getDLedgerStore().getLedgerEndIndex()); + assertEquals(stateMachine.getLastSnapshotIncludedIndex(), server.getDLedgerStore().getLedgerBeforeBeginIndex()); + // check statemachine + assertEquals(100, stateMachine.getTotalEntries()); + } + + Thread.sleep(100); + // Shutdown server + dLedgerServer0.shutdown(); + dLedgerServer1.shutdown(); + serverList.clear(); + // Restart server0 and server2 and apply snapshot, server1 simulated offline + dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + DLedgerServer dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer2); Thread.sleep(2000); // State machine could only be recovered from snapshot due to the entry has been removed after saving snapshot for (DLedgerServer server : serverList) { assertEquals(99, server.getDLedgerStore().getLedgerEndIndex()); - assertEquals(99, server.getDLedgerStore().getLedgerBeforeBeginIndex()); // check statemachine final MockStateMachine fsm = (MockStateMachine) server.getStateMachine(); assertEquals(100, fsm.getTotalEntries()); } + + // now keep appending entries + for (int i = 100; i < 200; i++) { + appendEntryResponse = dLedgerClient.append(new byte[512]); + assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); + assertEquals(i, appendEntryResponse.getIndex()); + } + Thread.sleep(2000); + for (DLedgerServer server : serverList) { + MockStateMachine stateMachine = (MockStateMachine) server.getStateMachine(); + assertEquals(199, server.getDLedgerStore().getLedgerEndIndex()); + assertEquals(stateMachine.getLastSnapshotIncludedIndex(), server.getDLedgerStore().getLedgerBeforeBeginIndex()); + // check statemachine + assertEquals(200, stateMachine.getTotalEntries()); + } + // shutdown server0 and server2 + dLedgerServer0.shutdown(); + dLedgerServer2.shutdown(); + serverList.clear(); + // restart all three servers, verify snapshot loading and verify if server1 catch up with cluster + dLedgerServer0 = launchServerWithStateMachineEnableSnapshot(group, peers, "n0", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer1 = launchServerWithStateMachineEnableSnapshot(group, peers, "n1", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + dLedgerServer2 = launchServerWithStateMachineEnableSnapshot(group, peers, "n2", "n0", DLedgerConfig.FILE, 10, 1024, new MockStateMachine()); + serverList.add(dLedgerServer0); + serverList.add(dLedgerServer1); + serverList.add(dLedgerServer2); + Thread.sleep(2000); + for (DLedgerServer server : serverList) { + assertEquals(199, server.getDLedgerStore().getLedgerEndIndex()); + // check statemachine + final MockStateMachine fsm = (MockStateMachine) server.getStateMachine(); + assertEquals(200, fsm.getTotalEntries()); + } } @Test @@ -119,7 +245,7 @@ public void testLoadErrorSnapshot() throws Exception { String group = UUID.randomUUID().toString(); String selfId = "n0"; String peers = String.format("%s-localhost:%d", selfId, nextPort()); - String snapshotBaseDirPrefix = FileTestUtil.TEST_BASE + File.separator + group + File.separator + "dledger-" + + String snapshotBaseDirPrefix = STORE_PATH + File.separator + group + File.separator + "dledger-" + selfId + File.separator + "snapshot" + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX; // Build error snapshot without state machine data @@ -144,7 +270,8 @@ public void testLoadErrorSnapshot() throws Exception { StateMachineCaller caller = server.getFsmCaller(); MockStateMachine fsm = (MockStateMachine) caller.getStateMachine(); - assertEquals(caller.getLastAppliedIndex(), 8); + MemberState memberState = server.getMemberState(); + assertEquals(memberState.getAppliedIndex(), 8); assertEquals(fsm.getTotalEntries(), 80); caller.shutdown(); } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotStoreTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotStoreTest.java index b3208416..51d61704 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotStoreTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotStoreTest.java @@ -15,21 +15,23 @@ public class SnapshotStoreTest { @Test public void testCreateReaderAndWriter() throws IOException { final long lastSnapshotIndex = 10; + String baseDir = null; try { - FileSnapshotStore writerStore = new FileSnapshotStore(FileTestUtil.TEST_BASE); + baseDir = FileTestUtil.createTestDir("SnapshotStoreTest"); + FileSnapshotStore writerStore = new FileSnapshotStore(baseDir); SnapshotWriter writer = writerStore.createSnapshotWriter(); Assertions.assertNotNull(writer); SnapshotMeta writerMeta = new SnapshotMeta(lastSnapshotIndex, 0); writer.setSnapshotMeta(writerMeta); writer.save(SnapshotStatus.SUCCESS); - FileSnapshotStore readerStore = new FileSnapshotStore(FileTestUtil.TEST_BASE); + FileSnapshotStore readerStore = new FileSnapshotStore(baseDir); SnapshotReader reader = readerStore.createSnapshotReader(); Assertions.assertNotNull(reader); SnapshotMeta readerMeta = reader.load(); Assertions.assertEquals(writerMeta.toString(), readerMeta.toString()); } finally { - IOUtils.deleteFile(new File(FileTestUtil.TEST_BASE + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex)); + IOUtils.deleteFile(new File(baseDir + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex)); } } } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/MockStateMachine.java b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/MockStateMachine.java index 2fa33bd8..eb870843 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/MockStateMachine.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/MockStateMachine.java @@ -28,8 +28,10 @@ public class MockStateMachine implements StateMachine { - private static Logger logger = LoggerFactory.getLogger(MockStateMachine.class); + private static final Logger logger = LoggerFactory.getLogger(MockStateMachine.class); private volatile long appliedIndex = -1L; + + private volatile long lastSnapshotIncludedIndex = -1L; private final AtomicLong totalEntries = new AtomicLong(0); @Override @@ -42,8 +44,8 @@ public void onApply(final CommittedEntryIterator iter) { } this.totalEntries.addAndGet(1); this.appliedIndex = next.getIndex(); - System.out.println("apply index: " + next.getIndex()); - System.out.println("total entries: " + this.totalEntries.get()); + logger.info("apply index: {}",this.appliedIndex); + logger.info("total entries: {}",this.totalEntries.get()); } } } @@ -51,8 +53,9 @@ public void onApply(final CommittedEntryIterator iter) { @Override public boolean onSnapshotSave(final SnapshotWriter writer) { long curEntryCnt = this.totalEntries.get(); + this.lastSnapshotIncludedIndex = this.appliedIndex; MockSnapshotFile snapshotFile = new MockSnapshotFile(writer.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); - System.out.println("save snapshot, total entries: " + curEntryCnt); + logger.info("save snapshot, lastIncludedIndex: {}, total entries: {}", this.lastSnapshotIncludedIndex, curEntryCnt); return snapshotFile.save(curEntryCnt); } @@ -63,9 +66,12 @@ public boolean onSnapshotLoad(final SnapshotReader reader) { File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); try { this.totalEntries.set(snapshotFile.load()); + this.appliedIndex = reader.getSnapshotMeta().getLastIncludedIndex(); + this.lastSnapshotIncludedIndex = this.appliedIndex; + logger.info("load snapshot, lastIncludedIndex: {}, total entries: {}", this.appliedIndex, this.totalEntries.get()); return true; } catch (IOException e) { - e.printStackTrace(); + logger.error("load snapshot failed", e); return false; } } @@ -92,4 +98,8 @@ public long getAppliedIndex() { public long getTotalEntries() { return this.totalEntries.get(); } + + public long getLastSnapshotIncludedIndex() { + return lastSnapshotIncludedIndex; + } } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/StateMachineCallerTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/StateMachineCallerTest.java index 2f8f07b9..65f80431 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/StateMachineCallerTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/StateMachineCallerTest.java @@ -48,6 +48,13 @@ class StateMachineCallerTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("StateMachineCallerTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } + private DLedgerConfig config; @Test @@ -92,6 +99,7 @@ public void testOnSnapshotLoad() throws Exception { final Pair result = mockCaller(dLedgerServer); final StateMachineCaller caller = result.getKey(); final MockStateMachine fsm = result.getValue(); + final MemberState memberState = dLedgerServer.getMemberState(); final long lastIncludedIndex = 10; String snapshotStoreBasePath = this.config.getSnapshotStoreBaseDir() + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastIncludedIndex; @@ -119,7 +127,7 @@ public void doCallBack(SnapshotStatus status) { } }); latch.await(); - assertEquals(caller.getLastAppliedIndex(), 10); + assertEquals(memberState.getAppliedIndex(), 10); assertEquals(fsm.getTotalEntries(), 90); caller.shutdown(); } @@ -127,7 +135,7 @@ public void doCallBack(SnapshotStatus status) { private DLedgerServer createDLedgerServerInStateMachineMode(String group, String peers, String selfId, String leaderId) { this.config = new DLedgerConfig(); this.config.group(group).selfId(selfId).peers(peers); - this.config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + this.config.setStoreBaseDir(STORE_PATH + File.separator + group); this.config.setSnapshotThreshold(0); this.config.setStoreType(DLedgerConfig.FILE); this.config.setMappedFileSizeForEntryData(10 * 1024 * 1024); @@ -167,7 +175,7 @@ private void updateFileStore(DLedgerMmapFileStore fileStore, int entryNum) { DLedgerEntry resEntry = fileStore.appendAsLeader(entry); assertEquals(i, resEntry.getIndex()); } - fileStore.updateCommittedIndex(memberState.currTerm(), entryNum - 1); + //fileStore.updateCommittedIndex(memberState.currTerm(), entryNum - 1); while (fileStore.getFlushPos() != fileStore.getWritePos()) { fileStore.flush(); } @@ -194,7 +202,7 @@ public void testServerWithStateMachine() throws InterruptedException { assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); assertEquals(i, appendEntryResponse.getIndex()); } - Thread.sleep(1200); + Thread.sleep(5000); for (DLedgerServer server : serverList) { assertEquals(9, server.getdLedgerStore().getLedgerEndIndex()); } diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadProcessor.java b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadProcessor.java index ef3f8ce9..1a3e96a3 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadProcessor.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadProcessor.java @@ -17,12 +17,11 @@ package io.openmessaging.storage.dledger.statemachine.register; import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.ReadClosure; -import io.openmessaging.storage.dledger.ReadMode; -import io.openmessaging.storage.dledger.Status; +import io.openmessaging.storage.dledger.common.ReadClosure; +import io.openmessaging.storage.dledger.common.ReadMode; +import io.openmessaging.storage.dledger.common.Status; import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; -import java.lang.reflect.Type; import java.util.concurrent.CompletableFuture; public class RegisterReadProcessor extends UserDefineProcessor { diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterStateMachine.java b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterStateMachine.java index 9f0e2444..08314653 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterStateMachine.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterStateMachine.java @@ -18,6 +18,7 @@ import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.exception.DLedgerException; +import io.openmessaging.storage.dledger.snapshot.SnapshotManager; import io.openmessaging.storage.dledger.snapshot.SnapshotReader; import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; import io.openmessaging.storage.dledger.statemachine.CommittedEntryIterator; @@ -25,6 +26,7 @@ import io.openmessaging.storage.dledger.statemachine.StateMachine; import io.openmessaging.storage.dledger.utils.BytesUtil; +import java.io.File; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -48,13 +50,13 @@ public void onApply(CommittedEntryIterator iter) { @Override public boolean onSnapshotSave(SnapshotWriter writer) { - RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(writer.getSnapshotStorePath()); + RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(writer.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); return registerSnapshotFile.save(this.register); } @Override public boolean onSnapshotLoad(SnapshotReader reader) { - RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(reader.getSnapshotStorePath()); + RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(reader.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); try{ Map register = registerSnapshotFile.load(); this.register = new ConcurrentHashMap<>(register); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerManagedFileStoreTruncateTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerManagedFileStoreTruncateTest.java index 6143436c..d4ce1cb8 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerManagedFileStoreTruncateTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerManagedFileStoreTruncateTest.java @@ -17,10 +17,17 @@ public class DLedgerManagedFileStoreTruncateTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("DLedgerManagedFileStoreTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } + private synchronized DLedgerMmapFileStore createFileStore(String group, String peers, String selfId, String leaderId, int dataFileSize, int indexFileSize, int deleteFileNums) { DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(STORE_PATH + File.separator + group); config.group(group).selfId(selfId).peers(peers); config.setStoreType(DLedgerConfig.MEMORY); config.setDiskSpaceRatioToForceClean(0.90f); diff --git a/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerMappedFileStoreTest.java b/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerMappedFileStoreTest.java index 45bf12a6..1961abf2 100644 --- a/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerMappedFileStoreTest.java +++ b/dledger/src/test/java/io/openmessaging/storage/dledger/store/DLedgerMappedFileStoreTest.java @@ -37,6 +37,13 @@ public class DLedgerMappedFileStoreTest extends ServerTestHarness { + public static final String STORE_PATH = FileTestUtil.createTestDir("DLedgerMappedFileStoreTest"); + + @Override + protected String getBaseDir() { + return STORE_PATH; + } + private synchronized DLedgerMmapFileStore createFileStore(String group, String peers, String selfId, String leaderId) { return createFileStore(group, peers, selfId, leaderId, 10 * 1024 * 1024, DLedgerMmapFileStore.INDEX_UNIT_SIZE * 1024 * 1024, 0); @@ -45,7 +52,7 @@ private synchronized DLedgerMmapFileStore createFileStore(String group, String p private synchronized DLedgerMmapFileStore createFileStore(String group, String peers, String selfId, String leaderId, int dataFileSize, int indexFileSize, int deleteFileNums) { DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); + config.setStoreBaseDir(STORE_PATH + File.separator + group); config.group(group).selfId(selfId).peers(peers); config.setStoreType(DLedgerConfig.MEMORY); config.setDiskSpaceRatioToForceClean(0.90f); @@ -87,32 +94,6 @@ private synchronized DLedgerMmapFileStore createFileStore(String group, String p return fileStore; } - @Test - public void testCommittedIndex() throws Exception { - String group = UUID.randomUUID().toString(); - String peers = String.format("n0-localhost:%d", nextPort()); - DLedgerMmapFileStore fileStore = createFileStore(group, peers, "n0", "n0"); - MemberState memberState = fileStore.getMemberState(); - for (int i = 0; i < 100; i++) { - DLedgerEntry entry = new DLedgerEntry(); - entry.setBody((new byte[128])); - DLedgerEntry resEntry = fileStore.appendAsLeader(entry); - Assertions.assertEquals(i, resEntry.getIndex()); - } - fileStore.updateCommittedIndex(memberState.currTerm(), 90); - Assertions.assertEquals(99, fileStore.getLedgerEndIndex()); - Assertions.assertEquals(90, fileStore.getCommittedIndex()); - - while (fileStore.getFlushPos() != fileStore.getWritePos()) { - fileStore.flush(); - } - fileStore.shutdown(); - fileStore = createFileStore(group, peers, "n0", "n0"); - Assertions.assertEquals(-1, fileStore.getLedgerBeforeBeginIndex()); - Assertions.assertEquals(99, fileStore.getLedgerEndIndex()); - Assertions.assertEquals(90, fileStore.getCommittedIndex()); - } - @Test public void testAppendHook() throws Exception { String group = UUID.randomUUID().toString(); @@ -151,22 +132,6 @@ public void testAppendAsLeader() throws Exception { Assertions.assertEquals(i, entry.getIndex()); Assertions.assertArrayEquals(("Hello Leader" + i).getBytes(), entry.getBody()); } - - for (long i = 0; i < 10; i++) { - fileStore.updateCommittedIndex(0, i); - Assertions.assertEquals(i, fileStore.getCommittedIndex()); - DLedgerEntry entry = fileStore.get(i); - Assertions.assertEquals(entry.getPos() + entry.getSize(), fileStore.getCommittedPos()); - } - Assertions.assertEquals(fileStore.getCommittedPos(), fileStore.getDataFileList().getMaxWrotePosition()); - - //ignore the smaller index and smaller term - fileStore.updateCommittedIndex(0, -1); - Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); - fileStore.updateCommittedIndex(0, 0); - Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); - fileStore.updateCommittedIndex(-1, 10); - Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); } @Test @@ -328,6 +293,92 @@ public void testResetOffsetAndRecover() { Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); } + /** + * Test reset offset to the end(clear all entries) and then recover + */ + @Test + public void testResetOffsetAndRecoverWithEmpty() { + String group = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d", nextPort()); + DLedgerMmapFileStore fileStore = createFileStore(group, peers, "n0", "n0", 1024, 1024, 0); + for (int i = 0; i < 10; i++) { + DLedgerEntry entry = new DLedgerEntry(); + // append an entry with 512 bytes body, total size is 512 + 48 = 560 bytes + // so every entry's size is 560 bytes + entry.setBody(new byte[512]); + DLedgerEntry resEntry = fileStore.appendAsLeader(entry); + Assertions.assertEquals(i, resEntry.getIndex()); + } + Assertions.assertEquals(10, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(-1, fileStore.getLedgerBeforeBeginIndex()); + Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); + + // reset offset, discard all 10 entries + DLedgerEntry entry = fileStore.get(9L); + long resetOffset = entry.getPos() + entry.getSize(); + fileStore.getDataFileList().resetOffset(resetOffset); + MmapFile firstMappedFile = fileStore.getDataFileList().getFirstMappedFile(); + Assertions.assertNotNull(firstMappedFile); + Assertions.assertEquals(1, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(560, firstMappedFile.getStartPosition()); + Assertions.assertEquals(560, firstMappedFile.getWrotePosition()); + ByteBuffer byteBuffer = firstMappedFile.sliceByteBuffer(); + int firstCode = byteBuffer.getInt(); + int firstSize = byteBuffer.getInt(); + Assertions.assertEquals(BLANK_MAGIC_CODE, firstCode); + Assertions.assertEquals(560, firstSize); + + // shutdown and restart + fileStore.shutdown(); + fileStore = createFileStore(group, peers, "n0", "n0", 1024, 1024, 0); + Assertions.assertEquals(1, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(-1, fileStore.getLedgerBeforeBeginIndex()); + Assertions.assertEquals(-1, fileStore.getLedgerEndIndex()); + } + + /** + * Test reset offset to the last entry(clear all entries excepted the last one) and then recover + */ + @Test + public void testResetOffsetAndRecoverWithEntry() { + String group = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d", nextPort()); + DLedgerMmapFileStore fileStore = createFileStore(group, peers, "n0", "n0", 1024, 1024, 0); + for (int i = 0; i < 10; i++) { + DLedgerEntry entry = new DLedgerEntry(); + // append an entry with 452 bytes body, total size is 452 + 48 = 500 bytes + // so every entry's size is 500 bytes + entry.setBody(new byte[452]); + DLedgerEntry resEntry = fileStore.appendAsLeader(entry); + Assertions.assertEquals(i, resEntry.getIndex()); + } + Assertions.assertEquals(5, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(-1, fileStore.getLedgerBeforeBeginIndex()); + Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); + + // reset offset, discard first 9 entries + DLedgerEntry entry = fileStore.get(8L); + long resetOffset = entry.getPos() + entry.getSize(); + fileStore.getDataFileList().resetOffset(resetOffset); + MmapFile firstMappedFile = fileStore.getDataFileList().getFirstMappedFile(); + Assertions.assertNotNull(firstMappedFile); + Assertions.assertEquals(1, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(500, firstMappedFile.getStartPosition()); + Assertions.assertEquals(1000, firstMappedFile.getWrotePosition()); + ByteBuffer byteBuffer = firstMappedFile.sliceByteBuffer(); + int firstCode = byteBuffer.getInt(); + int firstSize = byteBuffer.getInt(); + Assertions.assertEquals(BLANK_MAGIC_CODE, firstCode); + Assertions.assertEquals(500, firstSize); + + // shutdown and restart + fileStore.shutdown(); + fileStore = createFileStore(group, peers, "n0", "n0", 1024, 1024, 0); + Assertions.assertEquals(1, fileStore.getDataFileList().getMappedFiles().size()); + Assertions.assertEquals(8, fileStore.getLedgerBeforeBeginIndex()); + Assertions.assertEquals(9, fileStore.getLedgerEndIndex()); + } + @Test public void testAppendAsFollower() { DLedgerMmapFileStore fileStore = createFileStore(UUID.randomUUID().toString(), "n0-localhost:20913", "n0", "n1"); diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterStateMachine.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterStateMachine.java index a5505c4c..b571164b 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterStateMachine.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterStateMachine.java @@ -18,12 +18,14 @@ import io.openmessaging.storage.dledger.entry.DLedgerEntry; import io.openmessaging.storage.dledger.exception.DLedgerException; +import io.openmessaging.storage.dledger.snapshot.SnapshotManager; import io.openmessaging.storage.dledger.snapshot.SnapshotReader; import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; import io.openmessaging.storage.dledger.statemachine.CommittedEntryIterator; import io.openmessaging.storage.dledger.statemachine.StateMachine; import io.openmessaging.storage.dledger.utils.BytesUtil; +import java.io.File; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -47,13 +49,13 @@ public void onApply(CommittedEntryIterator iter) { @Override public boolean onSnapshotSave(SnapshotWriter writer) { - RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(writer.getSnapshotStorePath()); + RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(writer.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); return registerSnapshotFile.save(this.register); } @Override public boolean onSnapshotLoad(SnapshotReader reader) { - RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(reader.getSnapshotStorePath()); + RegisterSnapshotFile registerSnapshotFile = new RegisterSnapshotFile(reader.getSnapshotStorePath() + File.separator + SnapshotManager.SNAPSHOT_DATA_FILE); try { Map register = registerSnapshotFile.load(); this.register = new ConcurrentHashMap<>(register); diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/client/RegisterDLedgerClient.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/client/RegisterDLedgerClient.java index 72ebfaad..1e1eab88 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/client/RegisterDLedgerClient.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/client/RegisterDLedgerClient.java @@ -16,15 +16,19 @@ package io.openmessaging.storage.dledger.example.register.client; -import io.openmessaging.storage.dledger.ReadMode; +import io.openmessaging.storage.dledger.common.ReadMode; import io.openmessaging.storage.dledger.client.DLedgerClient; import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadRequest; import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadResponse; import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteRequest; import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class RegisterDLedgerClient { + private static final Logger LOGGER = LoggerFactory.getLogger(RegisterDLedgerClient.class); + private final DLedgerClient client; public RegisterDLedgerClient(String group, String peers) { @@ -48,6 +52,16 @@ public RegisterReadResponse read(int key) { return this.read(key, ReadMode.RAFT_LOG_READ); } + public RegisterReadResponse read(int key, String readModeStr) { + ReadMode readMode = ReadMode.RAFT_LOG_READ; + try { + readMode = ReadMode.valueOf(readModeStr); + } catch (Exception ignore) { + LOGGER.error("Error readMode string: {}, use default readMode: ", readModeStr, ReadMode.RAFT_LOG_READ); + } + return this.read(key, readMode); + } + public RegisterReadResponse read(int key, ReadMode readMode) { RegisterReadRequest request = new RegisterReadRequest(key, readMode); return client.invokeUserDefineRequest(request, RegisterReadResponse.class, true); diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/ReadCommand.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/ReadCommand.java index af1859e4..924cbbab 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/ReadCommand.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/ReadCommand.java @@ -19,7 +19,7 @@ import com.alibaba.fastjson.JSON; import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; -import io.openmessaging.storage.dledger.ReadMode; +import io.openmessaging.storage.dledger.common.ReadMode; import io.openmessaging.storage.dledger.example.common.command.BaseCommand; import io.openmessaging.storage.dledger.example.register.client.RegisterDLedgerClient; import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadResponse; @@ -40,7 +40,7 @@ public class ReadCommand extends BaseCommand { @Parameter(names = {"--key", "-k"}, description = "The key to read") private int key = 13; - @Parameter(names = {"--read-mode"}, description = "Read mode") + @Parameter(names = {"--read-mode", "-r"}, description = "Read mode") private ReadMode readMode = ReadMode.RAFT_LOG_READ; diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/RegisterCommand.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/RegisterCommand.java index 400ecf20..5dacaea9 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/RegisterCommand.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/command/RegisterCommand.java @@ -54,6 +54,21 @@ public class RegisterCommand extends BaseCommand { @Parameter(names = {"--preferred-leader-id"}, description = "Preferred LeaderId") private String preferredLeaderIds = null; + @Parameter(names = {"--enable-snapshot"}, description = "Enable snapshot") + private boolean enableSnapshot = false; + + @Parameter(names = {"--snapshot-threshold"}, description = "Snapshot threshold") + private int snapshotThreshold = 1000; + + @Parameter(names = {"--enable-batch-append"}, description = "Enable batch append") + private boolean enableBatchAppend = false; + + @Parameter(names = {"--max-batch-append-size"}, description = "Max batch append size") + private int maxBatchAppendSize = 4 * 1024; + + @Parameter(names = {"--max-batch-append-interval"}, description = "Max batch append interval in ms") + private int maxBatchAppendIntervalMs = 10; + @Override public void doCommand() { try { @@ -70,11 +85,16 @@ private DLedgerConfig buildDLedgerConfig() { dLedgerConfig.setGroup(this.group); dLedgerConfig.setSelfId(this.selfId); dLedgerConfig.setPeers(this.peers); - dLedgerConfig.setStoreBaseDir(this.storeBaseDir); + dLedgerConfig.setStoreBaseDir(this.storeBaseDir + File.separator + this.selfId); dLedgerConfig.setReadOnlyDataStoreDirs(this.readOnlyDataStoreDirs); dLedgerConfig.setPeerPushThrottlePoint(this.peerPushThrottlePoint); dLedgerConfig.setPeerPushQuota(this.peerPushQuota); dLedgerConfig.setPreferredLeaderIds(this.preferredLeaderIds); + dLedgerConfig.setEnableSnapshot(enableSnapshot); + dLedgerConfig.setSnapshotThreshold(snapshotThreshold); + dLedgerConfig.setEnableBatchAppend(enableBatchAppend); + dLedgerConfig.setMaxBatchAppendSize(maxBatchAppendSize); + dLedgerConfig.setMaxBatchAppendSize(maxBatchAppendIntervalMs); return dLedgerConfig; } } diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadProcessor.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadProcessor.java index b8e3fa5e..2a1b980c 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadProcessor.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadProcessor.java @@ -17,8 +17,8 @@ package io.openmessaging.storage.dledger.example.register.protocol; import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.ReadClosure; -import io.openmessaging.storage.dledger.Status; +import io.openmessaging.storage.dledger.common.ReadClosure; +import io.openmessaging.storage.dledger.common.Status; import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; import io.openmessaging.storage.dledger.example.register.RegisterStateMachine; import java.util.concurrent.CompletableFuture; diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadRequest.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadRequest.java index e35be7cf..0ee0c0de 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadRequest.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadRequest.java @@ -16,7 +16,7 @@ package io.openmessaging.storage.dledger.example.register.protocol; -import io.openmessaging.storage.dledger.ReadMode; +import io.openmessaging.storage.dledger.common.ReadMode; import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; public class RegisterReadRequest extends UserDefineRequest { @@ -50,4 +50,18 @@ public Integer getKey() { public int getRequestTypeCode() { return RegisterRequestTypeCode.READ.ordinal(); } + + @Override + public String toString() { + return "RegisterReadRequest{" + + "key=" + key + + ", readMode=" + readMode + + ", group='" + group + '\'' + + ", remoteId='" + remoteId + '\'' + + ", localId='" + localId + '\'' + + ", code=" + code + + ", leaderId='" + leaderId + '\'' + + ", term=" + term + + '}'; + } } diff --git a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteRequest.java b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteRequest.java index c624a722..83db679e 100644 --- a/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteRequest.java +++ b/example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteRequest.java @@ -18,7 +18,6 @@ import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; - public class RegisterWriteRequest extends UserDefineRequest { private Integer key; @@ -51,4 +50,17 @@ public int getRequestTypeCode() { return RegisterRequestTypeCode.WRITE.ordinal(); } + @Override + public String toString() { + return "RegisterWriteRequest{" + + "key=" + key + + ", value=" + value + + ", group='" + group + '\'' + + ", remoteId='" + remoteId + '\'' + + ", localId='" + localId + '\'' + + ", code=" + code + + ", leaderId='" + leaderId + '\'' + + ", term=" + term + + '}'; + } } diff --git a/jepsen/build.sh b/jepsen/build.sh index 512b2118..1b4c8cd4 100755 --- a/jepsen/build.sh +++ b/jepsen/build.sh @@ -18,7 +18,7 @@ # build dledger cd ../; mvn clean install -DskipTests; -mv example/target/dledger-example.jar jepsen/node-deploy/; +cp example/target/dledger-example.jar jepsen/node-deploy/; rm jepsen/dledger-jepsen.tar.gz; chmod +x jepsen/node-deploy/startup.sh; chmod +x jepsen/node-deploy/stop.sh; diff --git a/jepsen/src/main/clojure/io/openmessaging/storage/dledger/jepsen/core.clj b/jepsen/src/main/clojure/io/openmessaging/storage/dledger/jepsen/core.clj index b848e42b..0e312166 100644 --- a/jepsen/src/main/clojure/io/openmessaging/storage/dledger/jepsen/core.clj +++ b/jepsen/src/main/clojure/io/openmessaging/storage/dledger/jepsen/core.clj @@ -30,7 +30,7 @@ [jepsen.checker.timeline :as timeline] [knossos.model :as model]) (:import [io.openmessaging.storage.dledger.example.register.client RegisterDLedgerClient] - [io.openmessaging.storage.dledger ReadMode])) + [io.openmessaging.storage.dledger.common ReadMode])) (defonce dledger-path "/root/jepsen/node-deploy") (defonce dledger-port 20911) @@ -40,7 +40,9 @@ (defonce dledger-data-path "/tmp/dledgerstore") (defonce dledger-log-path "logs/dledger") -(defonce read-mode (atom ())) +(defonce read-mode (atom "RAFT_LOG_READ")) +(defonce enbale-snapshot (atom false)) +(defonce snapshot-threshold (atom 1000)) (defn peer-id [node] (str node)) @@ -66,7 +68,10 @@ "--id" (peer-id node) "--peers" - (peers test)))) + (peers test) + (if @enbale-snapshot + (str "--enable-snapshot --snapshot-threshold " @snapshot-threshold) + "")))) (defn stop! [node] (info "Stop DLedgerServer" node) @@ -104,7 +109,7 @@ "read a key-value from DLedger" [client key] (-> client :conn - (.read key))) + (.read key @read-mode))) (defn db "Regitser-Mode DLedger Server" @@ -162,6 +167,8 @@ (defn- parse-int [s] (Integer/parseInt s)) +(defn- parse-boolean [b] + (Boolean/parseBoolean b)) (defn r [_ _] {:type :invoke, :f :read, :value nil}) (defn w [_ _] {:type :invoke, :f :write, :value (rand-int 5)}) @@ -188,12 +195,19 @@ ["-p" "--passwd PWD" "SSH login user password." :default "passwd"] ["-m" "--read-mode MODE" "Read mode of DLedger." - :default "RAFT_LOG_READ" - :parse-fn parse-read-mode]]) + :default "RAFT_LOG_READ"] + [nil "--snapshot BOOL" "Whether to enbale snapshot mode." + :default false + :parse-fn parse-boolean] + [nil "--snapshot-threshold NUM" "Snapshot threshold." + :default 1000 + :parse-fn parse-int]]) (defn dledger-test [opts] (reset! read-mode (:read-mode opts)) + (reset! enbale-snapshot (:snapshot opts)) + (reset! snapshot-threshold (:snapshot-threshold opts)) (let [nemesis (get nemesis-map (:nemesis opts))] (merge tests/noop-test opts diff --git a/pom.xml b/pom.xml index 5e7b9b52..adb53d75 100644 --- a/pom.xml +++ b/pom.xml @@ -179,7 +179,7 @@ 1 1 - true + false **/IT*.java diff --git a/proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxy.java b/proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxy.java index b165e316..eb60a3a2 100644 --- a/proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxy.java +++ b/proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxy.java @@ -1,4 +1,4 @@ -/* +/** * Copyright 2017-2022 The DLedger Authors. * * Licensed under the Apache License, Version 2.0 (the "License"); @@ -15,14 +15,14 @@ */ package io.openmessaging.storage.dledger.proxy; -import io.openmessaging.storage.dledger.AppendFuture; +import io.openmessaging.storage.dledger.common.AppendFuture; import io.openmessaging.storage.dledger.DLedgerConfig; import io.openmessaging.storage.dledger.DLedgerRpcNettyService; import io.openmessaging.storage.dledger.DLedgerRpcService; import io.openmessaging.storage.dledger.DLedgerServer; import io.openmessaging.storage.dledger.AbstractDLedgerServer; -import io.openmessaging.storage.dledger.ReadClosure; -import io.openmessaging.storage.dledger.ReadMode; +import io.openmessaging.storage.dledger.common.ReadClosure; +import io.openmessaging.storage.dledger.common.ReadMode; import io.openmessaging.storage.dledger.exception.DLedgerException; import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; @@ -31,6 +31,8 @@ import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; +import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; import io.openmessaging.storage.dledger.protocol.LeadershipTransferRequest; import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; import io.openmessaging.storage.dledger.protocol.MetadataRequest; @@ -247,6 +249,11 @@ public CompletableFuture handlePush(PushEntryRequest request) } } + @Override + public CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) throws Exception { + return null; + } + public void startup() { this.dLedgerRpcService.startup(); this.dLedgerManager.startup(); diff --git a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyAppendAndGetTest.java b/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyAppendAndGetTest.java deleted file mode 100644 index 880dfe8b..00000000 --- a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyAppendAndGetTest.java +++ /dev/null @@ -1,404 +0,0 @@ -/* - * Copyright 2017-2022 The DLedger Authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.openmessaging.storage.dledger.proxy; - -import io.openmessaging.storage.dledger.DLedgerConfig; -import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.client.DLedgerClient; -import io.openmessaging.storage.dledger.entry.DLedgerEntry; -import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; -import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; -import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; -import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - - -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; - -/** - * Test the Multi-DLedger's append and get function - */ -public class ProxyAppendAndGetTest extends ServerTestHarness { - - @Test - public void testThreeServerInOneDLedgerProxyInMemory() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - List configs = new LinkedList<>(); - for (int i = 0; i < 3; i++) { - DLedgerConfig config = createDLedgerConfig(group, peers, "n" + i, "n1", DLedgerConfig.MEMORY); - configs.add(config); - } - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - DLedgerServer dLedgerServer0 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n0"); - DLedgerServer dLedgerServer1 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n1"); - DLedgerServer dLedgerServer2 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n2"); - Assertions.assertNotNull(dLedgerServer0); - Assertions.assertNotNull(dLedgerServer1); - Assertions.assertNotNull(dLedgerServer2); - Assertions.assertTrue(dLedgerServer1.getMemberState().isLeader()); - Thread.sleep(1000); - DLedgerClient dLedgerClient = launchClient(group, peers.split(";")[0]); - for (int i = 0; i < 10; i++) { - AppendEntryResponse appendEntryResponse = dLedgerClient.append(("HelloThreeServerInOneDLedgerProxyInMemory" + i).getBytes()); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); - Assertions.assertEquals(i, appendEntryResponse.getIndex()); - } - Thread.sleep(1000); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - for (int i = 0; i < 10; i++) { - GetEntriesResponse getEntriesResponse = dLedgerClient.get(i); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloThreeServerInOneDLedgerProxyInMemory" + i).getBytes(), getEntriesResponse.getEntries().get(0).getBody()); - } - } - - @Test - public void testThreeServerInOneDLedgerProxyInFile() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List configs = new LinkedList<>(); - for (int i = 0; i < 3; i++) { - DLedgerConfig config = createDLedgerConfig(group, peers, "n" + i, "n1", DLedgerConfig.FILE); - configs.add(config); - } - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - DLedgerServer dLedgerServer0 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n0"); - DLedgerServer dLedgerServer1 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n1"); - DLedgerServer dLedgerServer2 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n2"); - Assertions.assertNotNull(dLedgerServer0); - Assertions.assertNotNull(dLedgerServer1); - Assertions.assertNotNull(dLedgerServer2); - Assertions.assertTrue(dLedgerServer1.getMemberState().isLeader()); - DLedgerClient dLedgerClient = launchClient(group, peers); - for (int i = 0; i < 10; i++) { - AppendEntryResponse appendEntryResponse = dLedgerClient.append(("HelloThreeServerInOneDLedgerProxyInFile" + i).getBytes()); - Assertions.assertEquals(appendEntryResponse.getCode(), DLedgerResponseCode.SUCCESS.getCode()); - Assertions.assertEquals(i, appendEntryResponse.getIndex()); - } - Thread.sleep(100); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - for (int i = 0; i < 10; i++) { - GetEntriesResponse getEntriesResponse = dLedgerClient.get(i); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloThreeServerInOneDLedgerProxyInFile" + i).getBytes(), getEntriesResponse.getEntries().get(0).getBody()); - } - } - - @Test - public void testThreeServerInOneDLedgerProxyInFileWithAsyncRequests() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List configs = new LinkedList<>(); - for (int i = 0; i < 3; i++) { - DLedgerConfig config = createDLedgerConfig(group, peers, "n" + i, "n1", DLedgerConfig.FILE); - configs.add(config); - } - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - DLedgerServer dLedgerServer0 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n0"); - DLedgerServer dLedgerServer1 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n1"); - DLedgerServer dLedgerServer2 = dLedgerProxy.getDLedgerManager().getDLedgerServer(group, "n2"); - Assertions.assertNotNull(dLedgerServer0); - Assertions.assertNotNull(dLedgerServer1); - Assertions.assertNotNull(dLedgerServer2); - Assertions.assertTrue(dLedgerServer1.getMemberState().isLeader()); - List> futures = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - AppendEntryRequest request = new AppendEntryRequest(); - request.setGroup(group); - request.setRemoteId(dLedgerServer1.getMemberState().getSelfId()); - request.setBody(("HelloThreeServerInOneDLedgerProxyInFileWithAsyncRequests" + i).getBytes()); - futures.add(dLedgerProxy.handleAppend(request)); - } - Thread.sleep(1000); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - - DLedgerClient dLedgerClient = launchClient(group, peers); - for (int i = 0; i < futures.size(); i++) { - CompletableFuture future = futures.get(i); - Assertions.assertTrue(future.isDone()); - Assertions.assertEquals(i, future.get().getIndex()); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), future.get().getCode()); - - GetEntriesResponse getEntriesResponse = dLedgerClient.get(i); - DLedgerEntry entry = getEntriesResponse.getEntries().get(0); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloThreeServerInOneDLedgerProxyInFileWithAsyncRequests" + i).getBytes(), entry.getBody()); - //assert the pos - Assertions.assertEquals(entry.getPos(), future.get().getPos()); - } - } - - /** - * total three proxy, each proxy contains three servers and one of its servers will be a leader. - * - * @throws Exception - */ - @Test - public void testThreeProxyEachCarryThreeServerInMemory() throws Exception { - String group0 = UUID.randomUUID().toString(); - String group1 = UUID.randomUUID().toString(); - String group2 = UUID.randomUUID().toString(); - String[] groups = new String[]{group0, group1, group2}; - int port1 = nextPort(); - int port2 = nextPort(); - int port3 = nextPort(); - char[] prefix = new char[]{'a', 'b', 'c'}; - String peers0 = String.format("a0-localhost:%d;a1-localhost:%d;a2-localhost:%d", port1, port2, port3); - String peers1 = String.format("b0-localhost:%d;b1-localhost:%d;b2-localhost:%d", port1, port2, port3); - String peers2 = String.format("c0-localhost:%d;c1-localhost:%d;c2-localhost:%d", port1, port2, port3); - String[] peers = new String[]{peers0, peers1, peers2}; - DLedgerProxyConfig[] dLedgerProxyConfigs = new DLedgerProxyConfig[3]; - List[] configArr = new LinkedList[3]; - for (int i = 0; i < 3; i++) { - configArr[i] = new LinkedList<>(); - } - for (int i = 0; i < 3; i++) { - char selfPrefix = prefix[i]; - String leaderId = new String(new char[]{selfPrefix}) + i; - for (int j = 0; j < 3; j++) { - String selfId = new String(new char[]{selfPrefix}) + j; - DLedgerConfig config = createDLedgerConfig(groups[i], peers[i], selfId, leaderId, DLedgerConfig.MEMORY); - configArr[j].add(config); - } - } - for (int i = 0; i < 3; i++) { - dLedgerProxyConfigs[i] = new DLedgerProxyConfig(); - dLedgerProxyConfigs[i].setConfigs(configArr[i]); - } - System.out.println(dLedgerProxyConfigs); - DLedgerProxy[] proxies = launchDLedgerProxy(dLedgerProxyConfigs); - //get all leaders - DLedgerServer dLedgerServerA0 = proxies[0].getDLedgerManager().getDLedgerServer(group0, "a0"); - DLedgerServer dLedgerServerB1 = proxies[1].getDLedgerManager().getDLedgerServer(group1, "b1"); - DLedgerServer dLedgerServerC2 = proxies[2].getDLedgerManager().getDLedgerServer(group2, "c2"); - DLedgerServer[] servers = new DLedgerServer[]{dLedgerServerA0, dLedgerServerB1, dLedgerServerC2}; - Assertions.assertTrue(dLedgerServerA0.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerB1.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerC2.getMemberState().isLeader()); - DLedgerClient dLedgerClient0 = launchClient(group0, peers0.split(";")[0]); - DLedgerClient dLedgerClient1 = launchClient(group1, peers1.split(";")[0]); - DLedgerClient dLedgerClient2 = launchClient(group2, peers2.split(";")[0]); - DLedgerClient[] dLedgerClients = new DLedgerClient[]{dLedgerClient0, dLedgerClient1, dLedgerClient2}; - for (int k = 0; k < 3; k++) { - for (int i = 0; i < 10; i++) { - AppendEntryResponse appendEntryResponse = dLedgerClients[k].append(("HelloThreeProxyEachCarryThreeServerInMemory" + groups[k] + i).getBytes()); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); - Assertions.assertEquals(i, appendEntryResponse.getIndex()); - } - } - Thread.sleep(1000); - for (int k = 0; k < 3; k++) { - DLedgerServer dLedgerServer0 = proxies[k].getDLedgerManager().getDLedgerServer(groups[0], new String(new char[]{prefix[0]}) + k); - DLedgerServer dLedgerServer1 = proxies[k].getDLedgerManager().getDLedgerServer(groups[1], new String(new char[]{prefix[1]}) + k); - DLedgerServer dLedgerServer2 = proxies[k].getDLedgerManager().getDLedgerServer(groups[2], new String(new char[]{prefix[2]}) + k); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - } - - for (int k = 0; k < 3; k++) { - for (int i = 0; i < 10; i++) { - GetEntriesResponse getEntriesResponse = dLedgerClients[k].get(i); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloThreeProxyEachCarryThreeServerInMemory" + groups[k] + i).getBytes(), getEntriesResponse.getEntries().get(0).getBody()); - } - } - } - - @Test - public void testThreeProxyEachCarryThreeServerInFile() throws Exception { - String group0 = UUID.randomUUID().toString(); - String group1 = UUID.randomUUID().toString(); - String group2 = UUID.randomUUID().toString(); - String[] groups = new String[]{group0, group1, group2}; - int port1 = nextPort(); - int port2 = nextPort(); - int port3 = nextPort(); - char[] prefix = new char[]{'a', 'b', 'c'}; - String peers0 = String.format("a0-localhost:%d;a1-localhost:%d;a2-localhost:%d", port1, port2, port3); - String peers1 = String.format("b0-localhost:%d;b1-localhost:%d;b2-localhost:%d", port1, port2, port3); - String peers2 = String.format("c0-localhost:%d;c1-localhost:%d;c2-localhost:%d", port1, port2, port3); - String[] peers = new String[]{peers0, peers1, peers2}; - DLedgerProxyConfig[] dLedgerProxyConfigs = new DLedgerProxyConfig[3]; - List[] configArr = new LinkedList[3]; - for (int i = 0; i < 3; i++) { - configArr[i] = new LinkedList<>(); - } - for (int i = 0; i < 3; i++) { - char selfPrefix = prefix[i]; - String leaderId = new String(new char[]{selfPrefix}) + i; - for (int j = 0; j < 3; j++) { - String selfId = new String(new char[]{selfPrefix}) + j; - DLedgerConfig config = createDLedgerConfig(groups[i], peers[i], selfId, leaderId, DLedgerConfig.FILE); - configArr[j].add(config); - } - } - for (int i = 0; i < 3; i++) { - dLedgerProxyConfigs[i] = new DLedgerProxyConfig(); - dLedgerProxyConfigs[i].setConfigs(configArr[i]); - } - System.out.println(dLedgerProxyConfigs); - DLedgerProxy[] proxies = launchDLedgerProxy(dLedgerProxyConfigs); - //get all leaders - DLedgerServer dLedgerServerA0 = proxies[0].getDLedgerManager().getDLedgerServer(group0, "a0"); - DLedgerServer dLedgerServerB1 = proxies[1].getDLedgerManager().getDLedgerServer(group1, "b1"); - DLedgerServer dLedgerServerC2 = proxies[2].getDLedgerManager().getDLedgerServer(group2, "c2"); - DLedgerServer[] servers = new DLedgerServer[]{dLedgerServerA0, dLedgerServerB1, dLedgerServerC2}; - Assertions.assertTrue(dLedgerServerA0.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerB1.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerC2.getMemberState().isLeader()); - DLedgerClient dLedgerClient0 = launchClient(group0, peers0.split(";")[0]); - DLedgerClient dLedgerClient1 = launchClient(group1, peers1.split(";")[0]); - DLedgerClient dLedgerClient2 = launchClient(group2, peers2.split(";")[0]); - DLedgerClient[] dLedgerClients = new DLedgerClient[]{dLedgerClient0, dLedgerClient1, dLedgerClient2}; - for (int k = 0; k < 3; k++) { - for (int i = 0; i < 10; i++) { - AppendEntryResponse appendEntryResponse = dLedgerClients[k].append(("HelloThreeProxyEachCarryThreeServerInFile" + groups[k] + i).getBytes()); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); - Assertions.assertEquals(i, appendEntryResponse.getIndex()); - } - } - Thread.sleep(1000); - for (int k = 0; k < 3; k++) { - DLedgerServer dLedgerServer0 = proxies[k].getDLedgerManager().getDLedgerServer(groups[0], new String(new char[]{prefix[0]}) + k); - DLedgerServer dLedgerServer1 = proxies[k].getDLedgerManager().getDLedgerServer(groups[1], new String(new char[]{prefix[1]}) + k); - DLedgerServer dLedgerServer2 = proxies[k].getDLedgerManager().getDLedgerServer(groups[2], new String(new char[]{prefix[2]}) + k); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - } - - for (int k = 0; k < 3; k++) { - for (int i = 0; i < 10; i++) { - GetEntriesResponse getEntriesResponse = dLedgerClients[k].get(i); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloThreeProxyEachCarryThreeServerInFile" + groups[k] + i).getBytes(), getEntriesResponse.getEntries().get(0).getBody()); - } - } - } - - @Test - public void testTreeProxyEachCarryThreeServerInFileWithAsyncRequests() throws Exception { - String group0 = UUID.randomUUID().toString(); - String group1 = UUID.randomUUID().toString(); - String group2 = UUID.randomUUID().toString(); - String[] groups = new String[]{group0, group1, group2}; - int port1 = nextPort(); - int port2 = nextPort(); - int port3 = nextPort(); - char[] prefix = new char[]{'a', 'b', 'c'}; - String peers0 = String.format("a0-localhost:%d;a1-localhost:%d;a2-localhost:%d", port1, port2, port3); - String peers1 = String.format("b0-localhost:%d;b1-localhost:%d;b2-localhost:%d", port1, port2, port3); - String peers2 = String.format("c0-localhost:%d;c1-localhost:%d;c2-localhost:%d", port1, port2, port3); - String[] peers = new String[]{peers0, peers1, peers2}; - DLedgerProxyConfig[] dLedgerProxyConfigs = new DLedgerProxyConfig[3]; - List[] configArr = new LinkedList[3]; - for (int i = 0; i < 3; i++) { - configArr[i] = new LinkedList<>(); - } - for (int i = 0; i < 3; i++) { - char selfPrefix = prefix[i]; - String leaderId = new String(new char[]{selfPrefix}) + i; - for (int j = 0; j < 3; j++) { - String selfId = new String(new char[]{selfPrefix}) + j; - DLedgerConfig config = createDLedgerConfig(groups[i], peers[i], selfId, leaderId, DLedgerConfig.FILE); - configArr[j].add(config); - } - } - for (int i = 0; i < 3; i++) { - dLedgerProxyConfigs[i] = new DLedgerProxyConfig(); - dLedgerProxyConfigs[i].setConfigs(configArr[i]); - } - System.out.println(dLedgerProxyConfigs); - DLedgerProxy[] proxies = launchDLedgerProxy(dLedgerProxyConfigs); - //get all leaders - DLedgerServer dLedgerServerA0 = proxies[0].getDLedgerManager().getDLedgerServer(group0, "a0"); - DLedgerServer dLedgerServerB1 = proxies[1].getDLedgerManager().getDLedgerServer(group1, "b1"); - DLedgerServer dLedgerServerC2 = proxies[2].getDLedgerManager().getDLedgerServer(group2, "c2"); - DLedgerServer[] servers = new DLedgerServer[]{dLedgerServerA0, dLedgerServerB1, dLedgerServerC2}; - Assertions.assertTrue(dLedgerServerA0.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerB1.getMemberState().isLeader()); - Assertions.assertTrue(dLedgerServerC2.getMemberState().isLeader()); - DLedgerClient dLedgerClient0 = launchClient(group0, peers0.split(";")[0]); - DLedgerClient dLedgerClient1 = launchClient(group1, peers1.split(";")[0]); - DLedgerClient dLedgerClient2 = launchClient(group2, peers2.split(";")[0]); - DLedgerClient[] dLedgerClients = new DLedgerClient[]{dLedgerClient0, dLedgerClient1, dLedgerClient2}; - List> futures = new ArrayList<>(); - for (int k = 0; k < 3; k++) { - for (int i = 0; i < 10; i++) { - AppendEntryRequest request = new AppendEntryRequest(); - request.setGroup(groups[k]); - request.setRemoteId(servers[k].getMemberState().getSelfId()); - request.setBody(("HelloTreeProxyEachCarryThreeServerInFileWithAsyncRequests" + groups[k] + i).getBytes()); - futures.add(proxies[k].handleAppend(request)); - } - } - - Thread.sleep(1000); - for (int k = 0; k < 3; k++) { - DLedgerServer dLedgerServer0 = proxies[k].getDLedgerManager().getDLedgerServer(groups[0], new String(new char[]{prefix[0]}) + k); - DLedgerServer dLedgerServer1 = proxies[k].getDLedgerManager().getDLedgerServer(groups[1], new String(new char[]{prefix[1]}) + k); - DLedgerServer dLedgerServer2 = proxies[k].getDLedgerManager().getDLedgerServer(groups[2], new String(new char[]{prefix[2]}) + k); - Assertions.assertEquals(9, dLedgerServer0.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer1.getdLedgerStore().getLedgerEndIndex()); - Assertions.assertEquals(9, dLedgerServer2.getdLedgerStore().getLedgerEndIndex()); - } - - for (int k = 0; k < 3; k++) { - for (int i = 0; i < futures.size() / 3; i++) { - CompletableFuture future = futures.get(i); - Assertions.assertTrue(future.isDone()); - Assertions.assertEquals(i, future.get().getIndex()); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), future.get().getCode()); - GetEntriesResponse getEntriesResponse = dLedgerClients[k].get(i); - DLedgerEntry entry = getEntriesResponse.getEntries().get(0); - Assertions.assertEquals(1, getEntriesResponse.getEntries().size()); - Assertions.assertEquals(i, getEntriesResponse.getEntries().get(0).getIndex()); - Assertions.assertArrayEquals(("HelloTreeProxyEachCarryThreeServerInFileWithAsyncRequests" + groups[k] + i).getBytes(), entry.getBody()); - //assert the pos - Assertions.assertEquals(entry.getPos(), future.get().getPos()); - } - } - } - -} diff --git a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyLeaderElectorTest.java b/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyLeaderElectorTest.java deleted file mode 100644 index 9197780b..00000000 --- a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ProxyLeaderElectorTest.java +++ /dev/null @@ -1,309 +0,0 @@ -/* - * Copyright 2017-2022 The DLedger Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.openmessaging.storage.dledger.proxy; - -import io.openmessaging.storage.dledger.DLedgerConfig; -import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; -import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; -import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; -import io.openmessaging.storage.dledger.utils.DLedgerUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class ProxyLeaderElectorTest extends ServerTestHarness { - - @Test - public void testThreeServerInOneProxy() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List servers = new ArrayList<>(); - DLedgerConfig config0 = createDLedgerConfig(group, peers, "n0"); - DLedgerConfig config1 = createDLedgerConfig(group, peers, "n1"); - DLedgerConfig config2 = createDLedgerConfig(group, peers, "n2"); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(Arrays.asList(config0, config1, config2)); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - servers.addAll(dLedgerProxy.getDLedgerManager().getDLedgerServers()); - Thread.sleep(2000); - AtomicInteger leaderNum = new AtomicInteger(0); - AtomicInteger followerNum = new AtomicInteger(0); - DLedgerServer leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - - for (int i = 0; i < 10; i++) { - long maxTerm = servers.stream().max((o1, o2) -> { - if (o1.getMemberState().currTerm() < o2.getMemberState().currTerm()) { - return -1; - } else if (o1.getMemberState().currTerm() > o2.getMemberState().currTerm()) { - return 1; - } else { - return 0; - } - }).get().getMemberState().currTerm(); - DLedgerServer candidate = servers.get(i % servers.size()); - candidate.getdLedgerLeaderElector().testRevote(maxTerm + 1); - Thread.sleep(2000); - leaderNum.set(0); - followerNum.set(0); - leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - Assertions.assertTrue(candidate == leaderServer); - } - //write some data - for (int i = 0; i < 5; i++) { - AppendEntryRequest appendEntryRequest = new AppendEntryRequest(); - appendEntryRequest.setGroup(group); - appendEntryRequest.setRemoteId(leaderServer.getMemberState().getSelfId()); - appendEntryRequest.setBody("Hello Three Server In One Proxy".getBytes()); - AppendEntryResponse appendEntryResponse = leaderServer.getdLedgerRpcService().append(appendEntryRequest).get(); - Assertions.assertEquals(DLedgerResponseCode.SUCCESS.getCode(), appendEntryResponse.getCode()); - } - } - - @Test - public void testThreeServerInOneProxyAndRestartFollower() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List servers = new ArrayList<>(); - DLedgerConfig config0 = createDLedgerConfig(group, peers, "n0"); - DLedgerConfig config1 = createDLedgerConfig(group, peers, "n1"); - DLedgerConfig config2 = createDLedgerConfig(group, peers, "n2"); - List configs = Arrays.asList(config0, config1, config2); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(Arrays.asList(config0, config1, config2)); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - servers.addAll(dLedgerProxy.getDLedgerManager().getDLedgerServers()); - AtomicInteger leaderNum = new AtomicInteger(0); - AtomicInteger followerNum = new AtomicInteger(0); - long start = System.currentTimeMillis(); - while (parseServers(servers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 1000) { - Thread.sleep(100); - } - Thread.sleep(1000); - leaderNum.set(0); - followerNum.set(0); - DLedgerServer leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - - //restart the follower, the leader should keep the same - long term = leaderServer.getMemberState().currTerm(); - for (DLedgerServer server : servers) { - if (server == leaderServer) { - continue; - } - String followerId = server.getMemberState().getSelfId(); - dLedgerProxy.removeDLedgerServer(server.getMemberState().getGroup(), followerId); - dLedgerProxy.addDLedgerServer(server.getdLedgerConfig()); - Thread.sleep(2000); - Assertions.assertTrue(server.getMemberState().isFollower()); - Assertions.assertTrue(leaderServer.getMemberState().isLeader()); - Assertions.assertEquals(term, server.getMemberState().currTerm()); - } - } - - - @Test - public void testThreeServerInOneProxyAndShutdownLeader() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List servers = new ArrayList<>(); - DLedgerConfig config0 = createDLedgerConfig(group, peers, "n0"); - DLedgerConfig config1 = createDLedgerConfig(group, peers, "n1"); - DLedgerConfig config2 = createDLedgerConfig(group, peers, "n2"); - List configs = Arrays.asList(config0, config1, config2); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - servers.addAll(dLedgerProxy.getDLedgerManager().getDLedgerServers()); - AtomicInteger leaderNum = new AtomicInteger(0); - AtomicInteger followerNum = new AtomicInteger(0); - long start = System.currentTimeMillis(); - while (parseServers(servers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 1000) { - Thread.sleep(100); - } - Thread.sleep(1000); - leaderNum.set(0); - followerNum.set(0); - DLedgerServer leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - - //remove and shutdown the leader, should elect another leader - dLedgerProxy.removeDLedgerServer(leaderServer.getMemberState().getGroup(), leaderServer.getMemberState().getSelfId()); - Thread.sleep(1500); - List leftServers = new ArrayList<>(); - for (DLedgerServer server : servers) { - if (server != leaderServer) { - leftServers.add(server); - } - } - start = System.currentTimeMillis(); - while (parseServers(leftServers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 3 * leaderServer.getDLedgerConfig().getHeartBeatTimeIntervalMs()) { - Thread.sleep(100); - } - Thread.sleep(1000); - leaderNum.set(0); - followerNum.set(0); - Assertions.assertNotNull(parseServers(leftServers, leaderNum, followerNum)); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(1, followerNum.get()); - - } - - @Test - public void testThreeServerInOneProxyAndShutdownFollowers() throws Exception { - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List servers = new ArrayList<>(); - DLedgerConfig config0 = createDLedgerConfig(group, peers, "n0"); - DLedgerConfig config1 = createDLedgerConfig(group, peers, "n1"); - DLedgerConfig config2 = createDLedgerConfig(group, peers, "n2"); - List configs = Arrays.asList(config0, config1, config2); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - servers.addAll(dLedgerProxy.getDLedgerManager().getDLedgerServers()); - AtomicInteger leaderNum = new AtomicInteger(0); - AtomicInteger followerNum = new AtomicInteger(0); - - long start = System.currentTimeMillis(); - while (parseServers(servers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 1000) { - Thread.sleep(100); - } - Thread.sleep(1000); - leaderNum.set(0); - followerNum.set(0); - - DLedgerServer leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - - //shutdown the follower, the leader should keep the same - for (DLedgerServer server : servers) { - if (server == leaderServer) { - continue; - } - dLedgerProxy.removeDLedgerServer(server.getMemberState().getGroup(), server.getMemberState().getSelfId()); - } - - long term = leaderServer.getMemberState().currTerm(); - start = System.currentTimeMillis(); - while (leaderServer.getMemberState().isLeader() && DLedgerUtils.elapsed(start) < 4 * leaderServer.getdLedgerConfig().getHeartBeatTimeIntervalMs()) { - Thread.sleep(100); - } - Assertions.assertTrue(leaderServer.getMemberState().isCandidate()); - Assertions.assertEquals(term, leaderServer.getMemberState().currTerm()); - } - - @Test - public void testThreeServerInOneProxyAndPreferredLeader() throws Exception { - String preferredLeaderId = "n2"; - String group = UUID.randomUUID().toString(); - int port = nextPort(); - String peers = String.format("n0-localhost:%d;n1-localhost:%d;n2-localhost:%d", port, port, port); - List servers = new ArrayList<>(); - DLedgerConfig config0 = createDLedgerConfig(group, peers, "n0", preferredLeaderId); - DLedgerConfig config1 = createDLedgerConfig(group, peers, "n1", preferredLeaderId); - DLedgerConfig config2 = createDLedgerConfig(group, peers, "n2", preferredLeaderId); - List configs = Arrays.asList(config0, config1, config2); - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - dLedgerProxyConfig.setConfigs(configs); - DLedgerProxy dLedgerProxy = launchDLedgerProxy(dLedgerProxyConfig); - servers.addAll(dLedgerProxy.getDLedgerManager().getDLedgerServers()); - AtomicInteger leaderNum = new AtomicInteger(0); - AtomicInteger followerNum = new AtomicInteger(0); - - long start = System.currentTimeMillis(); - while (parseServers(servers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 1000) { - Thread.sleep(100); - } - Thread.sleep(3000); - leaderNum.set(0); - followerNum.set(0); - - DLedgerServer leaderServer = parseServers(servers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(2, followerNum.get()); - Assertions.assertNotNull(leaderServer); - - Assertions.assertEquals(preferredLeaderId, leaderServer.getdLedgerConfig().getSelfId()); - - //1.remove and shutdown leader. - dLedgerProxy.removeDLedgerServer(leaderServer.getMemberState().getGroup(), leaderServer.getMemberState().getSelfId()); - DLedgerConfig preferredLeaderConfig = leaderServer.getdLedgerConfig(); - Thread.sleep(1500); - List leftServers = new ArrayList<>(); - for (DLedgerServer server : servers) { - if (server != leaderServer) { - leftServers.add(server); - } - } - start = System.currentTimeMillis(); - while (parseServers(leftServers, leaderNum, followerNum) == null && DLedgerUtils.elapsed(start) < 3 * leaderServer.getdLedgerConfig().getHeartBeatTimeIntervalMs()) { - Thread.sleep(100); - } - Thread.sleep(300); - leaderNum.set(0); - followerNum.set(0); - leaderServer = parseServers(leftServers, leaderNum, followerNum); - Assertions.assertNotNull(leaderServer); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertEquals(1, followerNum.get()); - - - //2. restart leader; - long oldTerm = leaderServer.getMemberState().currTerm(); - dLedgerProxy.addDLedgerServer(preferredLeaderConfig); - Thread.sleep(500); - DLedgerServer newPreferredNode = dLedgerProxy.getDLedgerManager().getDLedgerServer(preferredLeaderConfig.getGroup(), preferredLeaderConfig.getSelfId()); - leftServers.add(newPreferredNode); - leaderNum.set(0); - followerNum.set(0); - start = System.currentTimeMillis(); - while (((leaderServer = parseServers(leftServers, leaderNum, followerNum)) == null || leaderServer.getMemberState().currTerm() == oldTerm) - && DLedgerUtils.elapsed(start) < 3000) { - Thread.sleep(100); - } - Thread.sleep(1500); - leaderNum.set(0); - followerNum.set(0); - leaderServer = parseServers(leftServers, leaderNum, followerNum); - Assertions.assertEquals(1, leaderNum.get()); - Assertions.assertTrue(followerNum.get() >= 1); - Assertions.assertNotNull(leaderServer); - Assertions.assertEquals(preferredLeaderId, leaderServer.getDLedgerConfig().getSelfId()); - } -} - diff --git a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestBase.java b/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestBase.java deleted file mode 100644 index da553f17..00000000 --- a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestBase.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2017-2022 The DLedger Authors - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.openmessaging.storage.dledger.proxy; -import io.openmessaging.storage.dledger.proxy.util.FileTestUtil; - -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; -import org.junit.jupiter.api.AfterEach; - -public class ServerTestBase { - - private static final AtomicInteger PORT_COUNTER = new AtomicInteger(30000); - private static Random random = new Random(); - protected List bases = new ArrayList<>(); - - public static int nextPort() { - return PORT_COUNTER.addAndGet(10 + random.nextInt(10)); - } - - @AfterEach - public synchronized void shutdown() { - for (String base : bases) { - try { - FileTestUtil.deleteFile(base); - } catch (Throwable ignored) { - - } - } - } -} diff --git a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestHarness.java b/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestHarness.java deleted file mode 100644 index e2e03070..00000000 --- a/proxy/src/test/java/io/openmessaging/storage/dledger/proxy/ServerTestHarness.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * Copyright 2017-2022 The DLedger Authors. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * https://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.openmessaging.storage.dledger.proxy; - -import io.openmessaging.storage.dledger.DLedgerConfig; -import io.openmessaging.storage.dledger.DLedgerServer; -import io.openmessaging.storage.dledger.MemberState; -import io.openmessaging.storage.dledger.client.DLedgerClient; -import io.openmessaging.storage.dledger.proxy.util.FileTestUtil; -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; - -public class ServerTestHarness extends ServerTestBase { - - protected synchronized DLedgerProxy launchDLedgerProxy(String group, String peers, String selfId) { - DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.group(group).selfId(selfId).peers(peers); - config.setStoreType(DLedgerConfig.MEMORY); - DLedgerProxy dLedgerProxy = new DLedgerProxy(config); - dLedgerProxy.startup(); - bases.add(config.getDefaultPath()); - return dLedgerProxy; - } - - protected synchronized DLedgerConfig createDLedgerConfig(String group, String peers, String selfId) { - DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.group(group).selfId(selfId).peers(peers); - config.setStoreType(DLedgerConfig.MEMORY); - bases.add(config.getDefaultPath()); - return config; - } - - - protected synchronized DLedgerProxy launchDLedgerProxy(String group, String peers, String selfId, String preferredLeaderId) { - DLedgerConfig config = createDLedgerConfig(group, peers, selfId, preferredLeaderId); - DLedgerProxy dLedgerProxy = new DLedgerProxy(config); - dLedgerProxy.startup(); - return dLedgerProxy; - } - - protected synchronized DLedgerConfig createDLedgerConfig(String group, String peers, String selfId, String preferredLeaderId) { - DLedgerConfig config = new DLedgerConfig(); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.group(group).selfId(selfId).peers(peers); - config.setStoreType(DLedgerConfig.MEMORY); - config.setPreferredLeaderIds(preferredLeaderId); - bases.add(config.getDefaultPath()); - return config; - } - - protected synchronized DLedgerProxy launchDLedgerProxy(String group, String peers, String selfId, String leaderId, - String storeType) { - DLedgerConfig config = new DLedgerConfig(); - config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.setStoreType(storeType); - config.setMappedFileSizeForEntryData(10 * 1024 * 1024); - config.setEnableLeaderElector(false); - config.setEnableDiskForceClean(false); - config.setDiskSpaceRatioToForceClean(0.90f); - DLedgerProxy dLedgerProxy = new DLedgerProxy(config); - MemberState memberState = dLedgerProxy.getDLedgerManager().getDLedgerServers().get(0).getMemberState(); - memberState.setCurrTermForTest(0); - if (selfId.equals(leaderId)) { - memberState.changeToLeader(0); - } else { - memberState.changeToFollower(0, leaderId); - } - bases.add(config.getDataStorePath()); - bases.add(config.getIndexStorePath()); - bases.add(config.getDefaultPath()); - dLedgerProxy.startup(); - return dLedgerProxy; - } - - protected synchronized DLedgerConfig createDLedgerConfig(String group, String peers, String selfId, String leaderId, - String storeType) { - DLedgerConfig config = new DLedgerConfig(); - config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.setStoreType(storeType); - config.setMappedFileSizeForEntryData(10 * 1024 * 1024); - config.setEnableLeaderElector(false); - config.setEnableDiskForceClean(false); - config.setDiskSpaceRatioToForceClean(0.90f); - config.setPreferredLeaderIds(leaderId); - bases.add(config.getDataStorePath()); - bases.add(config.getIndexStorePath()); - bases.add(config.getDefaultPath()); - return config; - } - - protected synchronized DLedgerProxy launchDLedgerProxy(DLedgerProxyConfig dLedgerProxyConfig) { - DLedgerProxy dLedgerProxy = new DLedgerProxy(dLedgerProxyConfig.getConfigs()); - for (DLedgerServer dLedgerServer : dLedgerProxy.getDLedgerManager().getDLedgerServers()) { - String leaderId = dLedgerServer.getdLedgerConfig().getPreferredLeaderIds(); - if (!dLedgerServer.getdLedgerConfig().isEnableLeaderElector() && leaderId != null) { - dLedgerServer.getMemberState().setCurrTermForTest(0); - if (dLedgerServer.getMemberState().getSelfId().equals(leaderId)) { - dLedgerServer.getMemberState().changeToLeader(0); - } else { - dLedgerServer.getMemberState().changeToFollower(0, leaderId); - } - } - } - dLedgerProxy.startup(); - return dLedgerProxy; - } - - protected synchronized DLedgerProxy[] launchDLedgerProxy(DLedgerProxyConfig[] dLedgerProxyConfigs) { - DLedgerProxy[] proxies = new DLedgerProxy[dLedgerProxyConfigs.length]; - for (int i = 0; i < dLedgerProxyConfigs.length; i++) { - DLedgerProxy dLedgerProxy = new DLedgerProxy(dLedgerProxyConfigs[i]); - for (DLedgerServer dLedgerServer : dLedgerProxy.getDLedgerManager().getDLedgerServers()) { - String leaderId = dLedgerServer.getdLedgerConfig().getPreferredLeaderIds(); - if (!dLedgerServer.getdLedgerConfig().isEnableLeaderElector() && leaderId != null) { - if (dLedgerServer.getMemberState().getSelfId().equals(leaderId)) { - dLedgerServer.getMemberState().changeToLeader(0); - } else { - dLedgerServer.getMemberState().changeToFollower(0, leaderId); - } - } - } - dLedgerProxy.startup(); - proxies[i] = dLedgerProxy; - } - return proxies; - } - - protected synchronized DLedgerServer launchServerEnableBatchPush(String group, String peers, String selfId, String leaderId, - String storeType) { - DLedgerConfig config = new DLedgerConfig(); - config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.setStoreType(storeType); - config.setMappedFileSizeForEntryData(10 * 1024 * 1024); - config.setEnableLeaderElector(false); - config.setEnableDiskForceClean(false); - config.setDiskSpaceRatioToForceClean(0.90f); - config.setEnableBatchPush(true); - config.setMaxBatchPushSize(300); - DLedgerServer dLedgerServer = new DLedgerServer(config); - MemberState memberState = dLedgerServer.getMemberState(); - memberState.setCurrTermForTest(0); - if (selfId.equals(leaderId)) { - memberState.changeToLeader(0); - } else { - memberState.changeToFollower(0, leaderId); - } - bases.add(config.getDataStorePath()); - bases.add(config.getIndexStorePath()); - bases.add(config.getDefaultPath()); - dLedgerServer.startup(); - return dLedgerServer; - } - - protected synchronized DLedgerProxy launchDLedgerProxyEnableBatchPush(String group, String peers, String selfId, String leaderId, - String storeType) { - DLedgerProxyConfig dLedgerProxyConfig = new DLedgerProxyConfig(); - DLedgerConfig config = new DLedgerConfig(); - config.group(group).selfId(selfId).peers(peers); - config.setStoreBaseDir(FileTestUtil.TEST_BASE + File.separator + group); - config.setStoreType(storeType); - config.setMappedFileSizeForEntryData(10 * 1024 * 1024); - config.setEnableLeaderElector(false); - config.setEnableDiskForceClean(false); - config.setDiskSpaceRatioToForceClean(0.90f); - config.setEnableBatchPush(true); - config.setMaxBatchPushSize(300); - dLedgerProxyConfig.setConfigs(Arrays.asList(config)); - DLedgerProxy dLedgerProxy = new DLedgerProxy(dLedgerProxyConfig); - MemberState memberState = dLedgerProxy.getDLedgerManager().getDLedgerServer(config.getGroup(), config.getSelfId()).getMemberState(); - memberState.setCurrTermForTest(0); - if (selfId.equals(leaderId)) { - memberState.changeToLeader(0); - } else { - memberState.changeToFollower(0, leaderId); - } - bases.add(config.getDataStorePath()); - bases.add(config.getIndexStorePath()); - bases.add(config.getDefaultPath()); - dLedgerProxy.startup(); - return dLedgerProxy; - } - - protected synchronized DLedgerClient launchClient(String group, String peers) { - DLedgerClient dLedgerClient = new DLedgerClient(group, peers); - dLedgerClient.startup(); - return dLedgerClient; - } - - protected DLedgerServer parseServers(List servers, AtomicInteger leaderNum, - AtomicInteger followerNum) { - DLedgerServer leaderServer = null; - for (DLedgerServer server : servers) { - if (server.getMemberState().isLeader()) { - leaderNum.incrementAndGet(); - leaderServer = server; - System.out.println("server:"+server.getdLedgerConfig().getSelfId()); - } else if (server.getMemberState().isFollower()) { - followerNum.incrementAndGet(); - } - } - return leaderServer; - } - - protected void simulatePartition(DLedgerServer server1, DLedgerServer server2) { - server1.getMemberState().getPeerMap().put(server2.getMemberState().getSelfId(), null); - server2.getMemberState().getPeerMap().put(server1.getMemberState().getSelfId(), null); - } -}