├── .github └── workflows │ └── maven.yml ├── .gitignore ├── LICENSE ├── README.md ├── dledger ├── pom.xml └── src │ ├── main │ ├── java │ │ └── io │ │ │ └── openmessaging │ │ │ └── storage │ │ │ └── dledger │ │ │ ├── AbstractDLedgerServer.java │ │ │ ├── DLedgerConfig.java │ │ │ ├── DLedgerEntryPusher.java │ │ │ ├── DLedgerLeaderElector.java │ │ │ ├── DLedgerRpcNettyService.java │ │ │ ├── DLedgerRpcService.java │ │ │ ├── DLedgerServer.java │ │ │ ├── MemberState.java │ │ │ ├── client │ │ │ ├── DLedgerClient.java │ │ │ ├── DLedgerClientRpcNettyService.java │ │ │ └── DLedgerClientRpcService.java │ │ │ ├── common │ │ │ ├── AppendFuture.java │ │ │ ├── BatchAppendFuture.java │ │ │ ├── Closure.java │ │ │ ├── NamedThreadFactory.java │ │ │ ├── ReadClosure.java │ │ │ ├── ReadMode.java │ │ │ ├── ShutdownAbleThread.java │ │ │ ├── Status.java │ │ │ ├── TimeoutFuture.java │ │ │ ├── WriteClosure.java │ │ │ └── WriteTask.java │ │ │ ├── entry │ │ │ ├── DLedgerEntry.java │ │ │ ├── DLedgerEntryCoder.java │ │ │ ├── DLedgerEntryType.java │ │ │ └── DLedgerIndexEntry.java │ │ │ ├── exception │ │ │ └── DLedgerException.java │ │ │ ├── metrics │ │ │ ├── DLedgerMetricsConstant.java │ │ │ ├── DLedgerMetricsManager.java │ │ │ ├── MetricsExporterType.java │ │ │ ├── NopLongCounter.java │ │ │ ├── NopLongHistogram.java │ │ │ └── NopObservableLongGauge.java │ │ │ ├── protocol │ │ │ ├── AppendEntryRequest.java │ │ │ ├── AppendEntryResponse.java │ │ │ ├── BatchAppendEntryRequest.java │ │ │ ├── DLedgerRequestCode.java │ │ │ ├── DLedgerResponseCode.java │ │ │ ├── GetEntriesRequest.java │ │ │ ├── GetEntriesResponse.java │ │ │ ├── HeartBeatRequest.java │ │ │ ├── HeartBeatResponse.java │ │ │ ├── InstallSnapshotRequest.java │ │ │ ├── InstallSnapshotResponse.java │ │ │ ├── LeadershipTransferRequest.java │ │ │ ├── LeadershipTransferResponse.java │ │ │ ├── MetadataRequest.java │ │ │ ├── MetadataResponse.java │ │ │ ├── PullEntriesRequest.java │ │ │ ├── PullEntriesResponse.java │ │ │ ├── PushEntryRequest.java │ │ │ ├── PushEntryResponse.java │ │ │ ├── RequestOrResponse.java │ │ │ ├── VoteRequest.java │ │ │ ├── VoteResponse.java │ │ │ ├── handler │ │ │ │ ├── DLedgerClientProtocolHandler.java │ │ │ │ ├── DLedgerInnerProtocolHandler.java │ │ │ │ ├── DLedgerProtocolHandler.java │ │ │ │ ├── DLedgerRaftProtocolHandler.java │ │ │ │ └── DLedgerRpcProtocolHandler.java │ │ │ ├── protocol │ │ │ │ ├── DLedgerClientProtocol.java │ │ │ │ ├── DLedgerProtocol.java │ │ │ │ └── DLedgerRaftProtocol.java │ │ │ └── userdefine │ │ │ │ ├── UserDefineCommandHeader.java │ │ │ │ ├── UserDefineProcessor.java │ │ │ │ ├── UserDefineRequest.java │ │ │ │ └── UserDefineResponse.java │ │ │ ├── snapshot │ │ │ ├── DownloadSnapshot.java │ │ │ ├── SnapshotEntryResetStrategy.java │ │ │ ├── SnapshotManager.java │ │ │ ├── SnapshotMeta.java │ │ │ ├── SnapshotReader.java │ │ │ ├── SnapshotStatus.java │ │ │ ├── SnapshotStore.java │ │ │ ├── SnapshotWriter.java │ │ │ ├── file │ │ │ │ ├── FileSnapshotReader.java │ │ │ │ ├── FileSnapshotStore.java │ │ │ │ └── FileSnapshotWriter.java │ │ │ └── hook │ │ │ │ ├── LoadSnapshotHook.java │ │ │ │ ├── SaveSnapshotHook.java │ │ │ │ └── SnapshotHook.java │ │ │ ├── statemachine │ │ │ ├── ApplyEntry.java │ │ │ ├── ApplyEntryIterator.java │ │ │ ├── NoOpStatemachine.java │ │ │ ├── StateMachine.java │ │ │ └── StateMachineCaller.java │ │ │ ├── store │ │ │ ├── DLedgerMemoryStore.java │ │ │ ├── DLedgerStore.java │ │ │ └── file │ │ │ │ ├── DLedgerMmapFileStore.java │ │ │ │ ├── DefaultMmapFile.java │ │ │ │ ├── MmapFile.java │ │ │ │ ├── MmapFileList.java │ │ │ │ ├── MultiPathMmapFileList.java │ │ │ │ ├── ReferenceResource.java │ │ │ │ └── SelectMmapBufferResult.java │ │ │ └── utils │ │ │ ├── BytesUtil.java │ │ │ ├── DLedgerUtils.java │ │ │ ├── IOUtils.java │ │ │ ├── Pair.java │ │ │ ├── PreConditions.java │ │ │ ├── Quota.java │ │ │ └── ResettableCountDownLatch.java │ └── resources │ │ ├── config.example.yaml │ │ └── config.yaml │ └── test │ └── java │ └── io │ └── openmessaging │ └── storage │ └── dledger │ ├── AppendAndGetTest.java │ ├── AppendAndPushTest.java │ ├── AppendAndReadTest.java │ ├── BatchPushTest.java │ ├── CommitIndexTest.java │ ├── LeaderElectorTest.java │ ├── ServerTestBase.java │ ├── ServerTestHarness.java │ ├── WriteTest.java │ ├── common │ └── NamedThreadFactoryTest.java │ ├── entry │ └── DLedgerEntryCoderTest.java │ ├── protocol │ ├── HeartbeatRequestTest.java │ └── VoteRequestTest.java │ ├── snapshot │ ├── SnapshotManagerTest.java │ ├── SnapshotReaderTest.java │ ├── SnapshotStoreTest.java │ └── SnapshotWriterTest.java │ ├── statemachine │ ├── MockSnapshotFile.java │ ├── MockStateMachine.java │ ├── RegisterSnapshotFile.java │ ├── StateMachineCallerTest.java │ └── register │ │ ├── RegisterReadProcessor.java │ │ ├── RegisterReadRequest.java │ │ ├── RegisterReadResponse.java │ │ └── RegisterStateMachine.java │ ├── store │ ├── DLedgerManagedFileStoreTruncateTest.java │ ├── DLedgerMappedFileStoreTest.java │ ├── MmapFileListTest.java │ └── file │ │ └── DefaultMmapFileTest.java │ └── util │ ├── FileTestUtil.java │ └── QuotaTest.java ├── example ├── pom.xml └── src │ ├── main │ ├── java │ │ └── io │ │ │ └── openmessaging │ │ │ └── storage │ │ │ └── dledger │ │ │ └── example │ │ │ ├── CommandCli.java │ │ │ ├── appender │ │ │ ├── AppenderDLedger.java │ │ │ └── command │ │ │ │ ├── AppendCommand.java │ │ │ │ ├── AppenderCommand.java │ │ │ │ ├── ConfigCommand.java │ │ │ │ └── GetCommand.java │ │ │ ├── common │ │ │ └── command │ │ │ │ ├── BaseCommand.java │ │ │ │ ├── LeadershipTransferCommand.java │ │ │ │ └── ReadFileCommand.java │ │ │ └── register │ │ │ ├── RegisterBenchmark.java │ │ │ ├── RegisterDLedger.java │ │ │ ├── RegisterSnapshotFile.java │ │ │ ├── RegisterStateMachine.java │ │ │ ├── client │ │ │ └── RegisterDLedgerClient.java │ │ │ ├── command │ │ │ ├── BenchmarkCommand.java │ │ │ ├── ReadCommand.java │ │ │ ├── RegisterCommand.java │ │ │ └── WriteCommand.java │ │ │ └── protocol │ │ │ ├── RegisterReadProcessor.java │ │ │ ├── RegisterReadRequest.java │ │ │ ├── RegisterReadResponse.java │ │ │ ├── RegisterRequestTypeCode.java │ │ │ ├── RegisterWriteProcessor.java │ │ │ ├── RegisterWriteRequest.java │ │ │ └── RegisterWriteResponse.java │ └── resources │ │ └── appender │ │ ├── config.example.error.yaml │ │ └── config.example.yaml │ └── test │ └── java │ └── io │ └── openmessaging │ └── storage │ └── dledger │ └── example │ └── appender │ └── CommandTest.java ├── jepsen ├── build.sh ├── build_deploy.sh ├── control.clj ├── jepsen.sh ├── node-deploy │ ├── startup.sh │ ├── stop.sh │ └── stop_dropcaches.sh ├── nodes ├── project.clj └── src │ └── main │ └── clojure │ └── io │ └── openmessaging │ └── storage │ └── dledger │ └── jepsen │ └── core.clj ├── pom.xml ├── proxy ├── pom.xml └── src │ ├── main │ └── java │ │ └── io │ │ └── openmessaging │ │ └── storage │ │ └── dledger │ │ └── proxy │ │ ├── ConfigManager.java │ │ ├── DLedgerManager.java │ │ ├── DLedgerProxy.java │ │ ├── DLedgerProxyConfig.java │ │ ├── DLedgerProxyConfigListener.java │ │ └── util │ │ └── ConfigUtils.java │ └── test │ ├── java │ └── io │ │ └── openmessaging │ │ └── storage │ │ └── dledger │ │ └── proxy │ │ └── util │ │ ├── ConfigUtilsTest.java │ │ └── FileTestUtil.java │ └── resources │ ├── config.example.error.yaml │ └── config.example.yaml ├── script └── deploy-three-register.sh └── style ├── copyright ├── Apache.xml └── profiles_settings.xml ├── dledger_checkstyle.xml └── dledger_codestyle.xml /.github/workflows/maven.yml: -------------------------------------------------------------------------------- 1 | # This workflow will build a Java project with Maven, and cache/restore any dependencies to improve the workflow execution time 2 | # For more information see: https://help.github.com/actions/language-and-framework-guides/building-and-testing-java-with-maven 3 | 4 | name: Java CI with Maven 5 | 6 | on: 7 | push: 8 | branches: [ "master" ] 9 | pull_request: 10 | types: [ opened, reopened, synchronize ] 11 | branches: [ "master" ] 12 | 13 | jobs: 14 | build: 15 | 16 | runs-on: ubuntu-latest 17 | 18 | steps: 19 | - uses: actions/checkout@v3 20 | - name: Set up JDK 8 21 | uses: actions/setup-java@v3 22 | with: 23 | java-version: '8' 24 | distribution: 'temurin' 25 | cache: maven 26 | - name: Build with Maven 27 | run: mvn -B package --file pom.xml -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *dependency-reduced-pom.xml 2 | .classpath 3 | .project 4 | .settings/ 5 | target/ 6 | localbin/ 7 | devenv 8 | *.log* 9 | *.iml 10 | .idea/ 11 | *.versionsBackup 12 | !NOTICE-BIN 13 | !LICENSE-BIN 14 | .DS_Store 15 | nohup.out 16 | *.gz 17 | dledger-example.jar 18 | .clj-kondo 19 | *.lsp 20 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/AbstractDLedgerServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger; 18 | 19 | import io.openmessaging.storage.dledger.protocol.handler.DLedgerProtocolHandler; 20 | import org.apache.rocketmq.remoting.netty.NettyRemotingClient; 21 | import org.apache.rocketmq.remoting.netty.NettyRemotingServer; 22 | 23 | public abstract class AbstractDLedgerServer implements DLedgerProtocolHandler { 24 | 25 | public abstract String getListenAddress(); 26 | 27 | public abstract String getPeerAddr(String groupId, String selfId); 28 | 29 | public abstract NettyRemotingServer getRemotingServer(); 30 | 31 | public abstract NettyRemotingClient getRemotingClient(); 32 | } 33 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/DLedgerRpcService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger; 18 | 19 | import io.openmessaging.storage.dledger.protocol.handler.DLedgerRpcProtocolHandler; 20 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; 21 | import io.openmessaging.storage.dledger.protocol.protocol.DLedgerProtocol; 22 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; 23 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineResponse; 24 | 25 | public abstract class DLedgerRpcService implements DLedgerProtocol, DLedgerRpcProtocolHandler { 26 | 27 | public abstract void startup(); 28 | 29 | public abstract void shutdown(); 30 | 31 | public abstract void registerUserDefineProcessor(UserDefineProcessor userDefineProcessor); 32 | 33 | } 34 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/client/DLedgerClientRpcService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.client; 18 | 19 | import io.openmessaging.storage.dledger.protocol.protocol.DLedgerClientProtocol; 20 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; 21 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineResponse; 22 | 23 | import java.util.Map; 24 | import java.util.concurrent.CompletableFuture; 25 | import java.util.concurrent.ConcurrentHashMap; 26 | 27 | public abstract class DLedgerClientRpcService implements DLedgerClientProtocol { 28 | private final Map peerMap = new ConcurrentHashMap<>(); 29 | 30 | public void updatePeers(String peers) { 31 | for (String peerInfo : peers.split(";")) { 32 | String nodeId = peerInfo.split("-")[0]; 33 | peerMap.put(nodeId, peerInfo.substring(nodeId.length() + 1)); 34 | } 35 | } 36 | 37 | public void updatePeers(Map peers) { 38 | peerMap.putAll(peers); 39 | } 40 | 41 | public String getPeerAddr(String id) { 42 | return peerMap.get(id); 43 | } 44 | 45 | public abstract void startup(); 46 | 47 | public abstract void shutdown(); 48 | 49 | public abstract CompletableFuture invokeUserDefineRequest(T request, Class aClass) throws Exception; 50 | } 51 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/AppendFuture.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | public class AppendFuture extends TimeoutFuture { 20 | 21 | private long pos = -1; 22 | 23 | public AppendFuture() { 24 | 25 | } 26 | 27 | public AppendFuture(long timeOutMs) { 28 | this.timeOutMs = timeOutMs; 29 | } 30 | 31 | public long getPos() { 32 | return pos; 33 | } 34 | 35 | public void setPos(long pos) { 36 | this.pos = pos; 37 | } 38 | 39 | public static AppendFuture newCompletedFuture(long pos, T value) { 40 | AppendFuture future = new AppendFuture(); 41 | future.setPos(pos); 42 | future.complete(value); 43 | return future; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/BatchAppendFuture.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package io.openmessaging.storage.dledger.common; 17 | 18 | public class BatchAppendFuture extends AppendFuture { 19 | private long[] positions; 20 | 21 | public BatchAppendFuture() { 22 | 23 | } 24 | 25 | public BatchAppendFuture(long[] positions) { 26 | this.positions = positions; 27 | } 28 | 29 | public BatchAppendFuture(long timeOutMs) { 30 | super(timeOutMs); 31 | } 32 | 33 | public long[] getPositions() { 34 | return positions; 35 | } 36 | 37 | public void setPositions(long[] positions) { 38 | this.positions = positions; 39 | } 40 | 41 | public static BatchAppendFuture newCompletedFuture(long pos, T value) { 42 | BatchAppendFuture future = new BatchAppendFuture(); 43 | future.setPos(pos); 44 | future.complete(value); 45 | return future; 46 | } 47 | } -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/Closure.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | public abstract class Closure { 20 | 21 | protected long createTime = System.currentTimeMillis(); 22 | 23 | protected long timeoutMs = 2000; 24 | 25 | public Closure() { 26 | 27 | } 28 | 29 | public Closure(long timeoutMs) { 30 | this.timeoutMs = timeoutMs; 31 | } 32 | 33 | public boolean isTimeOut() { 34 | return System.currentTimeMillis() - createTime >= timeoutMs; 35 | } 36 | 37 | public abstract void done(Status status); 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/NamedThreadFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | import java.util.concurrent.ThreadFactory; 20 | import java.util.concurrent.atomic.AtomicInteger; 21 | 22 | public class NamedThreadFactory implements ThreadFactory { 23 | 24 | private final AtomicInteger threadIndex; 25 | 26 | private final String threadNamePrefix; 27 | 28 | private final boolean isDaemonThread; 29 | 30 | public NamedThreadFactory(final String threadNamePrefix, boolean isDaemonThread) { 31 | this(new AtomicInteger(0), threadNamePrefix, isDaemonThread); 32 | } 33 | 34 | public NamedThreadFactory(AtomicInteger threadIndex, final String threadNamePrefix, boolean isDaemonThread) { 35 | this.threadIndex = threadIndex; 36 | this.threadNamePrefix = threadNamePrefix; 37 | this.isDaemonThread = isDaemonThread; 38 | } 39 | 40 | public NamedThreadFactory(final String threadNamePrefix) { 41 | this(threadNamePrefix, false); 42 | } 43 | 44 | /** 45 | * Constructs a new {@code Thread}. Implementations may also initialize priority, name, daemon status, {@code 46 | * ThreadGroup}, etc. 47 | * 48 | * @param r a runnable to be executed by new thread instance 49 | * @return constructed thread, or {@code null} if the request to create a thread is rejected 50 | */ 51 | @Override 52 | public Thread newThread(Runnable r) { 53 | 54 | StringBuilder threadName = new StringBuilder(threadNamePrefix); 55 | if (null != threadIndex) { 56 | threadName.append("-").append(threadIndex.incrementAndGet()); 57 | } 58 | Thread thread = new Thread(r, threadName.toString()); 59 | if (isDaemonThread) { 60 | thread.setDaemon(true); 61 | } 62 | return thread; 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadClosure.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | public abstract class ReadClosure extends Closure { 20 | 21 | public abstract void done(Status status); 22 | 23 | } 24 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/ReadMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | public enum ReadMode { 20 | 21 | UNSAFE_READ, 22 | RAFT_LOG_READ, 23 | READ_INDEX_READ, 24 | LEASE_READ 25 | 26 | } 27 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/Status.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; 20 | 21 | public class Status { 22 | 23 | public DLedgerResponseCode code = DLedgerResponseCode.SUCCESS; 24 | private Status() { 25 | 26 | } 27 | 28 | private Status(DLedgerResponseCode code) { 29 | this.code = code; 30 | } 31 | 32 | public boolean isOk() { 33 | return this.code.getCode() == 200; 34 | } 35 | 36 | public static Status ok() { 37 | return new Status(); 38 | } 39 | 40 | public static Status error(DLedgerResponseCode code) { 41 | return new Status(code); 42 | } 43 | 44 | @Override 45 | public String toString() { 46 | return "Status{" + 47 | "code=" + code + 48 | '}'; 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/TimeoutFuture.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | import java.util.concurrent.CompletableFuture; 20 | 21 | public class TimeoutFuture extends CompletableFuture { 22 | 23 | protected long createTimeMs = System.currentTimeMillis(); 24 | 25 | protected long timeOutMs = 1000; 26 | 27 | public TimeoutFuture() { 28 | 29 | } 30 | 31 | public TimeoutFuture(long timeOutMs) { 32 | this.timeOutMs = timeOutMs; 33 | } 34 | 35 | public long getCreateTimeMs() { 36 | return createTimeMs; 37 | } 38 | 39 | public void setCreateTimeMs(long createTimeMs) { 40 | this.createTimeMs = createTimeMs; 41 | } 42 | 43 | public long getTimeOutMs() { 44 | return timeOutMs; 45 | } 46 | 47 | public void setTimeOutMs(long timeOutMs) { 48 | this.timeOutMs = timeOutMs; 49 | } 50 | 51 | public boolean isTimeOut() { 52 | return System.currentTimeMillis() - createTimeMs >= timeOutMs; 53 | } 54 | 55 | } 56 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/WriteClosure.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | public abstract class WriteClosure extends Closure { 20 | public abstract void setResp(T t); 21 | 22 | public abstract T getResp(); 23 | } 24 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/common/WriteTask.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | 20 | public class WriteTask { 21 | 22 | private byte[] body; 23 | 24 | public void setBody(byte[] body) { 25 | this.body = body; 26 | } 27 | 28 | public byte[] getBody() { 29 | return body; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerEntryType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.entry; 18 | 19 | /** 20 | * DLedgerEntryType, upper layer can only see the NORMAL entry, the other two are used internally. 21 | */ 22 | public enum DLedgerEntryType { 23 | 24 | /** 25 | * The entry which contains the upper layer business data. 26 | */ 27 | NORMAL(1), 28 | 29 | /** 30 | * The entry with empty body, used for RaftLog-Read and commit index fast advanced when the leader is changed. 31 | */ 32 | NOOP(2), 33 | 34 | /** 35 | * TODO: The entry used for configuration change. 36 | */ 37 | CONFIG_CHANGE(3); 38 | 39 | int magic; 40 | 41 | DLedgerEntryType(int magic) { 42 | this.magic = magic; 43 | } 44 | 45 | public int getMagic() { 46 | return magic; 47 | } 48 | 49 | public static boolean isValid(int magic) { 50 | return magic == NORMAL.getMagic() || magic == NOOP.getMagic() || magic == CONFIG_CHANGE.getMagic(); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/entry/DLedgerIndexEntry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.entry; 18 | 19 | public class DLedgerIndexEntry { 20 | 21 | private int magic; 22 | 23 | private long position; 24 | 25 | private int size; 26 | 27 | private long index; 28 | 29 | private long term; 30 | 31 | public int getMagic() { 32 | return magic; 33 | } 34 | 35 | public void setMagic(int magic) { 36 | this.magic = magic; 37 | } 38 | 39 | public long getPosition() { 40 | return position; 41 | } 42 | 43 | public void setPosition(long position) { 44 | this.position = position; 45 | } 46 | 47 | public int getSize() { 48 | return size; 49 | } 50 | 51 | public void setSize(int size) { 52 | this.size = size; 53 | } 54 | 55 | public long getIndex() { 56 | return index; 57 | } 58 | 59 | public void setIndex(long index) { 60 | this.index = index; 61 | } 62 | 63 | public long getTerm() { 64 | return term; 65 | } 66 | 67 | public void setTerm(long term) { 68 | this.term = term; 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/exception/DLedgerException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.exception; 18 | 19 | import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; 20 | 21 | public class DLedgerException extends RuntimeException { 22 | 23 | private final DLedgerResponseCode code; 24 | 25 | public DLedgerException(DLedgerResponseCode code, String message) { 26 | super(message); 27 | this.code = code; 28 | } 29 | 30 | public DLedgerException(DLedgerResponseCode code, String format, Object... args) { 31 | super(String.format(format, args)); 32 | this.code = code; 33 | } 34 | 35 | public DLedgerResponseCode getCode() { 36 | return code; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/metrics/DLedgerMetricsConstant.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.metrics; 18 | 19 | public class DLedgerMetricsConstant { 20 | 21 | // metrics name 22 | 23 | public static final String HISTOGRAM_APPEND_ENTRY_LATENCY = "dledger_append_entry_latency"; 24 | 25 | public static final String HISTOGRAM_APPEND_ENTRY_BATCH_BYTES = "dledger_append_entry_batch_bytes"; 26 | 27 | public static final String HISTOGRAM_APPEND_ENTRY_BATCH_COUNT = "dledger_append_entry_batch_count"; 28 | 29 | public static final String HISTOGRAM_REPLICATE_ENTRY_LATENCY = "dledger_replicate_entry_latency"; 30 | 31 | public static final String HISTOGRAM_REPLICATE_ENTRY_BATCH_BYTES = "dledger_replicate_entry_batch_bytes"; 32 | 33 | public static final String HISTOGRAM_REPLICATE_ENTRY_BATCH_COUNT = "dledger_replicate_entry_batch_count"; 34 | 35 | public static final String HISTOGRAM_APPLY_TASK_LATENCY = "dledger_apply_task_latency"; 36 | 37 | public static final String HISTOGRAM_APPLY_TASK_BATCH_COUNT = "dledger_apply_task_batch_count"; 38 | 39 | public static final String HISTOGRAM_READ_LATENCY = "dledger_read_latency"; 40 | 41 | public static final String HISTOGRAM_SAVE_SNAPSHOT_LATENCY = "dledger_save_snapshot_latency"; 42 | 43 | public static final String HISTOGRAM_LOAD_SNAPSHOT_LATENCY = "dledger_load_snapshot_latency"; 44 | 45 | public static final String HISTOGRAM_INSTALL_SNAPSHOT_LATENCY = "dledger_install_snapshot_latency"; 46 | 47 | public static final String GAUGE_ENTRIES_COUNT = "dledger_entries_count"; 48 | 49 | public static final String GAUGE_SNAPSHOT_COUNT = "dledger_snapshot_count"; 50 | 51 | public static final String GAUGE_ENTRY_STORE_SIZE = "dledger_entry_store_size"; 52 | 53 | 54 | // label 55 | 56 | public static final String LABEL_GROUP = "group"; 57 | 58 | public static final String LABEL_SELF_ID = "self_id"; 59 | 60 | public static final String LABEL_REMOTE_ID = "remote_id"; 61 | 62 | public static final String LABEL_READ_MODE = "read_mode"; 63 | 64 | public static final String METER_NAME = "dledger"; 65 | 66 | } 67 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/metrics/MetricsExporterType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.metrics; 18 | 19 | 20 | public enum MetricsExporterType { 21 | DISABLE(0), 22 | OTLP_GRPC(1), 23 | PROM(2), 24 | LOG(3); 25 | 26 | private final int value; 27 | 28 | MetricsExporterType(int value) { 29 | this.value = value; 30 | } 31 | 32 | public int getValue() { 33 | return this.value; 34 | } 35 | 36 | public static MetricsExporterType valueOf(int value) { 37 | switch (value) { 38 | case 1: 39 | return OTLP_GRPC; 40 | case 2: 41 | return PROM; 42 | case 3: 43 | return LOG; 44 | default: 45 | return DISABLE; 46 | } 47 | } 48 | 49 | public boolean isEnable() { 50 | return this.value > 0; 51 | } 52 | } -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/metrics/NopLongCounter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.metrics; 18 | 19 | import io.opentelemetry.api.common.Attributes; 20 | import io.opentelemetry.api.metrics.LongCounter; 21 | import io.opentelemetry.context.Context; 22 | 23 | public class NopLongCounter implements LongCounter { 24 | @Override 25 | public void add(long l) { 26 | 27 | } 28 | 29 | @Override 30 | public void add(long l, Attributes attributes) { 31 | 32 | } 33 | 34 | @Override 35 | public void add(long l, Attributes attributes, Context context) { 36 | 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/metrics/NopLongHistogram.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.metrics; 18 | 19 | import io.opentelemetry.api.common.Attributes; 20 | import io.opentelemetry.api.metrics.LongHistogram; 21 | import io.opentelemetry.context.Context; 22 | 23 | public class NopLongHistogram implements LongHistogram { 24 | 25 | @Override 26 | public void record(long l) { 27 | 28 | } 29 | 30 | @Override 31 | public void record(long l, Attributes attributes) { 32 | 33 | } 34 | 35 | @Override 36 | public void record(long l, Attributes attributes, Context context) { 37 | 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/metrics/NopObservableLongGauge.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.metrics; 18 | 19 | import io.opentelemetry.api.metrics.ObservableLongGauge; 20 | 21 | public class NopObservableLongGauge implements ObservableLongGauge { 22 | } 23 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/AppendEntryRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class AppendEntryRequest extends RequestOrResponse { 20 | 21 | private byte[] body; 22 | 23 | public byte[] getBody() { 24 | return body; 25 | } 26 | 27 | public void setBody(byte[] body) { 28 | this.body = body; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/AppendEntryResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class AppendEntryResponse extends RequestOrResponse { 20 | 21 | private long index = -1; 22 | private long pos = -1; 23 | 24 | public long getIndex() { 25 | return index; 26 | } 27 | 28 | public void setIndex(long index) { 29 | this.index = index; 30 | } 31 | 32 | public long getPos() { 33 | return pos; 34 | } 35 | 36 | public void setPos(long pos) { 37 | this.pos = pos; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/BatchAppendEntryRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import java.util.List; 20 | 21 | public class BatchAppendEntryRequest extends AppendEntryRequest { 22 | private List batchMsgs; 23 | 24 | public List getBatchMsgs() { 25 | return batchMsgs; 26 | } 27 | 28 | public void setBatchMsgs(List batchMsgs) { 29 | this.batchMsgs = batchMsgs; 30 | } 31 | } -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/DLedgerRequestCode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import java.util.HashMap; 20 | import java.util.Map; 21 | 22 | public enum DLedgerRequestCode { 23 | UNKNOWN(-1, ""), 24 | METADATA(50000, ""), 25 | APPEND(50001, ""), 26 | GET(50002, ""), 27 | VOTE(51001, ""), 28 | HEART_BEAT(51002, ""), 29 | PULL(51003, ""), 30 | PUSH(51004, ""), 31 | LEADERSHIP_TRANSFER(51005, ""), 32 | 33 | INSTALL_SNAPSHOT(51006, ""), 34 | USER_DEFINE_REQUEST(59999, ""); 35 | 36 | 37 | private static Map codeMap = new HashMap<>(); 38 | 39 | static { 40 | for (DLedgerRequestCode requestCode : DLedgerRequestCode.values()) { 41 | codeMap.put(requestCode.code, requestCode); 42 | } 43 | } 44 | 45 | private int code; 46 | private String desc; 47 | 48 | DLedgerRequestCode(int code, String desc) { 49 | this.code = code; 50 | this.desc = desc; 51 | } 52 | 53 | public static DLedgerRequestCode valueOf(int code) { 54 | DLedgerRequestCode tmp = codeMap.get(code); 55 | if (tmp != null) { 56 | return tmp; 57 | } else { 58 | return UNKNOWN; 59 | } 60 | 61 | } 62 | 63 | public int getCode() { 64 | return code; 65 | } 66 | 67 | public String getDesc() { 68 | return desc; 69 | } 70 | 71 | @Override 72 | public String toString() { 73 | return String.format("[code=%d,name=%s,desc=%s]", code, name(), desc); 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/GetEntriesRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import java.util.List; 20 | 21 | public class GetEntriesRequest extends RequestOrResponse { 22 | private Long beginIndex; 23 | 24 | private int maxSize; 25 | 26 | private List indexList; 27 | 28 | public Long getBeginIndex() { 29 | return beginIndex; 30 | } 31 | 32 | public void setBeginIndex(Long beginIndex) { 33 | this.beginIndex = beginIndex; 34 | } 35 | 36 | public int getMaxSize() { 37 | return maxSize; 38 | } 39 | 40 | public void setMaxSize(int maxSize) { 41 | this.maxSize = maxSize; 42 | } 43 | 44 | public List getIndexList() { 45 | return indexList; 46 | } 47 | 48 | public void setIndexList(List indexList) { 49 | this.indexList = indexList; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/GetEntriesResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import io.openmessaging.storage.dledger.entry.DLedgerEntry; 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | 23 | public class GetEntriesResponse extends RequestOrResponse { 24 | private List entries = new ArrayList<>(); 25 | 26 | public List getEntries() { 27 | return entries; 28 | } 29 | 30 | public void setEntries(List entries) { 31 | this.entries = entries; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/HeartBeatRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class HeartBeatRequest extends RequestOrResponse { 20 | 21 | } 22 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/HeartBeatResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class HeartBeatResponse extends RequestOrResponse { 20 | 21 | public HeartBeatResponse term(long term) { 22 | this.term = term; 23 | return this; 24 | } 25 | 26 | @Override 27 | public HeartBeatResponse code(int code) { 28 | this.code = code; 29 | return this; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotRequest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class InstallSnapshotRequest extends RequestOrResponse { 20 | 21 | private long lastIncludedIndex; 22 | 23 | private long lastIncludedTerm; 24 | 25 | private byte[] data; 26 | 27 | public InstallSnapshotRequest() { 28 | } 29 | 30 | public InstallSnapshotRequest(long lastIncludedIndex, long lastIncludedTerm, byte[] data) { 31 | this.lastIncludedIndex = lastIncludedIndex; 32 | this.lastIncludedTerm = lastIncludedTerm; 33 | this.data = data; 34 | } 35 | 36 | public long getLastIncludedIndex() { 37 | return lastIncludedIndex; 38 | } 39 | 40 | public void setLastIncludedIndex(long lastIncludedIndex) { 41 | this.lastIncludedIndex = lastIncludedIndex; 42 | } 43 | 44 | public long getLastIncludedTerm() { 45 | return lastIncludedTerm; 46 | } 47 | 48 | public void setLastIncludedTerm(long lastIncludedTerm) { 49 | this.lastIncludedTerm = lastIncludedTerm; 50 | } 51 | 52 | public byte[] getData() { 53 | return data; 54 | } 55 | 56 | public void setData(byte[] data) { 57 | this.data = data; 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "InstallSnapshotRequest{" + 63 | "lastIncludedIndex=" + lastIncludedIndex + 64 | ", lastIncludedTerm=" + lastIncludedTerm + 65 | '}'; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/InstallSnapshotResponse.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class InstallSnapshotResponse extends RequestOrResponse { 20 | 21 | public InstallSnapshotResponse() { 22 | } 23 | 24 | @Override 25 | public RequestOrResponse code(int code) { 26 | this.code = code; 27 | return this; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/LeadershipTransferRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class LeadershipTransferRequest extends RequestOrResponse { 20 | 21 | private String transferId; 22 | private String transfereeId; 23 | private long takeLeadershipLedgerIndex; 24 | 25 | public String getTransfereeId() { 26 | return transfereeId; 27 | } 28 | 29 | public void setTransfereeId(String transfereeId) { 30 | this.transfereeId = transfereeId; 31 | } 32 | 33 | public String getTransferId() { 34 | return transferId; 35 | } 36 | 37 | public void setTransferId(String transferId) { 38 | this.transferId = transferId; 39 | } 40 | 41 | public long getTakeLeadershipLedgerIndex() { 42 | return takeLeadershipLedgerIndex; 43 | } 44 | 45 | public void setTakeLeadershipLedgerIndex(long takeLeadershipLedgerIndex) { 46 | this.takeLeadershipLedgerIndex = takeLeadershipLedgerIndex; 47 | } 48 | 49 | @Override 50 | public String toString() { 51 | return "LeadershipTransferRequest{" + 52 | "transferId='" + transferId + '\'' + 53 | ", transfereeId='" + transfereeId + '\'' + 54 | ", takeLeadershipLedgerIndex=" + takeLeadershipLedgerIndex + 55 | ", group='" + group + '\'' + 56 | ", remoteId='" + remoteId + '\'' + 57 | ", localId='" + localId + '\'' + 58 | ", code=" + code + 59 | ", leaderId='" + leaderId + '\'' + 60 | ", term=" + term + 61 | '}'; 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/LeadershipTransferResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class LeadershipTransferResponse extends RequestOrResponse { 20 | 21 | public LeadershipTransferResponse term(long term) { 22 | this.term = term; 23 | return this; 24 | } 25 | 26 | @Override 27 | public LeadershipTransferResponse code(int code) { 28 | this.code = code; 29 | return this; 30 | } 31 | 32 | @Override 33 | public String toString() { 34 | return "LeadershipTransferResponse{" + 35 | "group='" + group + '\'' + 36 | ", remoteId='" + remoteId + '\'' + 37 | ", localId='" + localId + '\'' + 38 | ", code=" + code + 39 | ", leaderId='" + leaderId + '\'' + 40 | ", term=" + term + 41 | '}'; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/MetadataRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class MetadataRequest extends RequestOrResponse { 20 | 21 | } 22 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/MetadataResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import java.util.Map; 20 | 21 | public class MetadataResponse extends RequestOrResponse { 22 | 23 | private Map peers; 24 | 25 | public Map getPeers() { 26 | return peers; 27 | } 28 | 29 | public void setPeers(Map peers) { 30 | this.peers = peers; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PullEntriesRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class PullEntriesRequest extends RequestOrResponse { 20 | private String nodeId; 21 | private Long beginIndex; 22 | 23 | public Long getBeginIndex() { 24 | return beginIndex; 25 | } 26 | 27 | public void setBeginIndex(Long beginIndex) { 28 | this.beginIndex = beginIndex; 29 | } 30 | 31 | public String getNodeId() { 32 | return nodeId; 33 | } 34 | 35 | public void setNodeId(String nodeId) { 36 | this.nodeId = nodeId; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PullEntriesResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import io.openmessaging.storage.dledger.entry.DLedgerEntry; 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | 23 | public class PullEntriesResponse extends RequestOrResponse { 24 | private List entries = new ArrayList<>(); 25 | 26 | public List getEntries() { 27 | return entries; 28 | } 29 | 30 | public void setEntries(List entries) { 31 | this.entries = entries; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/PushEntryResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class PushEntryResponse extends RequestOrResponse { 20 | private Long index; 21 | 22 | private long beginIndex; 23 | private long endIndex; 24 | private int count; 25 | 26 | private long xTerm = -1; 27 | 28 | private long xIndex = -1; 29 | 30 | public long getXTerm() { 31 | return xTerm; 32 | } 33 | 34 | public void setXTerm(long xTerm) { 35 | this.xTerm = xTerm; 36 | } 37 | 38 | public long getXIndex() { 39 | return xIndex; 40 | } 41 | 42 | public void setXIndex(long xIndex) { 43 | this.xIndex = xIndex; 44 | } 45 | 46 | public Long getIndex() { 47 | return index; 48 | } 49 | 50 | public void setIndex(Long index) { 51 | this.index = index; 52 | } 53 | 54 | public long getBeginIndex() { 55 | return beginIndex; 56 | } 57 | 58 | public void setBeginIndex(long beginIndex) { 59 | this.beginIndex = beginIndex; 60 | } 61 | 62 | public long getEndIndex() { 63 | return endIndex; 64 | } 65 | 66 | public void setEndIndex(long endIndex) { 67 | this.endIndex = endIndex; 68 | } 69 | 70 | public int getCount() { 71 | return count; 72 | } 73 | 74 | public void setCount(int count) { 75 | this.count = count; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/RequestOrResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class RequestOrResponse { 20 | 21 | protected String group; 22 | protected String remoteId; 23 | protected String localId; 24 | 25 | protected int code = DLedgerResponseCode.SUCCESS.getCode(); 26 | 27 | protected String leaderId = null; 28 | 29 | protected long term = -1; 30 | 31 | public String getGroup() { 32 | return group; 33 | } 34 | 35 | public void setGroup(String group) { 36 | this.group = group; 37 | } 38 | 39 | public int getCode() { 40 | return code; 41 | } 42 | 43 | public void setCode(int code) { 44 | this.code = code; 45 | } 46 | 47 | public RequestOrResponse code(int code) { 48 | this.code = code; 49 | return this; 50 | } 51 | 52 | public void setIds(String localId, String remoteId, String leaderId) { 53 | this.localId = localId; 54 | this.remoteId = remoteId; 55 | this.leaderId = leaderId; 56 | } 57 | 58 | public String getRemoteId() { 59 | return remoteId; 60 | } 61 | 62 | public void setRemoteId(String remoteId) { 63 | this.remoteId = remoteId; 64 | } 65 | 66 | public String getLocalId() { 67 | return localId; 68 | } 69 | 70 | public void setLocalId(String localId) { 71 | this.localId = localId; 72 | } 73 | 74 | public String getLeaderId() { 75 | return leaderId; 76 | } 77 | 78 | public void setLeaderId(String leaderId) { 79 | this.leaderId = leaderId; 80 | } 81 | 82 | public long getTerm() { 83 | return term; 84 | } 85 | 86 | public void setTerm(long term) { 87 | this.term = term; 88 | } 89 | 90 | public RequestOrResponse copyBaseInfo(RequestOrResponse other) { 91 | this.group = other.group; 92 | this.term = other.term; 93 | this.code = other.code; 94 | this.localId = other.localId; 95 | this.remoteId = other.remoteId; 96 | this.leaderId = other.leaderId; 97 | return this; 98 | } 99 | 100 | public String baseInfo() { 101 | return String.format("info[group=%s,term=%d,code=%d,local=%s,remote=%s,leader=%s]", group, term, code, localId, remoteId, leaderId); 102 | } 103 | 104 | } 105 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/VoteRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | public class VoteRequest extends RequestOrResponse { 20 | 21 | private long ledgerEndIndex = -1; 22 | 23 | private long ledgerEndTerm = -1; 24 | 25 | public long getLedgerEndIndex() { 26 | return ledgerEndIndex; 27 | } 28 | 29 | public void setLedgerEndIndex(long ledgerEndIndex) { 30 | this.ledgerEndIndex = ledgerEndIndex; 31 | } 32 | 33 | public long getLedgerEndTerm() { 34 | return ledgerEndTerm; 35 | } 36 | 37 | public void setLedgerEndTerm(long ledgerEndTerm) { 38 | this.ledgerEndTerm = ledgerEndTerm; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/VoteResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol; 18 | 19 | import static io.openmessaging.storage.dledger.protocol.VoteResponse.RESULT.UNKNOWN; 20 | 21 | public class VoteResponse extends RequestOrResponse { 22 | 23 | public RESULT voteResult = UNKNOWN; 24 | 25 | public VoteResponse() { 26 | 27 | } 28 | 29 | public VoteResponse(VoteRequest request) { 30 | copyBaseInfo(request); 31 | } 32 | 33 | public RESULT getVoteResult() { 34 | return voteResult; 35 | } 36 | 37 | public void setVoteResult(RESULT voteResult) { 38 | this.voteResult = voteResult; 39 | } 40 | 41 | public VoteResponse voteResult(RESULT voteResult) { 42 | this.voteResult = voteResult; 43 | return this; 44 | } 45 | 46 | public VoteResponse term(long term) { 47 | this.term = term; 48 | return this; 49 | } 50 | 51 | public enum RESULT { 52 | UNKNOWN, 53 | ACCEPT, 54 | REJECT_UNKNOWN_LEADER, 55 | REJECT_UNEXPECTED_LEADER, 56 | REJECT_EXPIRED_VOTE_TERM, 57 | REJECT_ALREADY_VOTED, 58 | REJECT_ALREADY_HAS_LEADER, 59 | REJECT_TERM_NOT_READY, 60 | REJECT_TERM_SMALL_THAN_LEDGER, 61 | REJECT_EXPIRED_LEDGER_TERM, 62 | REJECT_SMALL_LEDGER_END_INDEX, 63 | REJECT_TAKING_LEADERSHIP; 64 | } 65 | 66 | public enum ParseResult { 67 | WAIT_TO_REVOTE, 68 | REVOTE_IMMEDIATELY, 69 | PASSED, 70 | WAIT_TO_VOTE_NEXT; 71 | } 72 | 73 | @Override 74 | public String toString() { 75 | return "VoteResponse{" + 76 | "group='" + group + '\'' + 77 | ", remoteId='" + remoteId + '\'' + 78 | ", localId='" + localId + '\'' + 79 | ", code=" + code + 80 | ", leaderId='" + leaderId + '\'' + 81 | ", term=" + term + 82 | ", voteResult=" + voteResult + 83 | '}'; 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerClientProtocolHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.handler; 18 | 19 | 20 | import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; 21 | import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; 22 | import io.openmessaging.storage.dledger.protocol.GetEntriesRequest; 23 | import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; 24 | import io.openmessaging.storage.dledger.protocol.LeadershipTransferRequest; 25 | import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; 26 | import io.openmessaging.storage.dledger.protocol.MetadataRequest; 27 | import io.openmessaging.storage.dledger.protocol.MetadataResponse; 28 | 29 | import java.util.concurrent.CompletableFuture; 30 | 31 | /** 32 | * Both the RaftLogServer(inbound) and RaftRpcService (outbound) should implement this protocol 33 | */ 34 | public interface DLedgerClientProtocolHandler { 35 | 36 | CompletableFuture handleAppend(AppendEntryRequest request) throws Exception; 37 | 38 | CompletableFuture handleGet(GetEntriesRequest request) throws Exception; 39 | 40 | CompletableFuture handleMetadata(MetadataRequest request) throws Exception; 41 | 42 | CompletableFuture handleLeadershipTransfer(LeadershipTransferRequest leadershipTransferRequest) throws Exception; 43 | } 44 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerInnerProtocolHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.handler; 18 | 19 | import io.openmessaging.storage.dledger.common.ReadClosure; 20 | import io.openmessaging.storage.dledger.common.ReadMode; 21 | import io.openmessaging.storage.dledger.common.WriteClosure; 22 | import io.openmessaging.storage.dledger.common.WriteTask; 23 | 24 | public interface DLedgerInnerProtocolHandler { 25 | 26 | void handleRead(ReadMode mode, ReadClosure closure); 27 | 28 | void handleWrite(WriteTask task, WriteClosure closure); 29 | 30 | } 31 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerProtocolHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.handler; 18 | 19 | 20 | /** 21 | * Both the RaftLogServer(inbound) and RaftRpcService (outbound) should implement this protocol 22 | */ 23 | public interface DLedgerProtocolHandler extends DLedgerRpcProtocolHandler, DLedgerInnerProtocolHandler { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRaftProtocolHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.handler; 18 | 19 | import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; 20 | import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; 21 | import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; 22 | import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; 23 | import io.openmessaging.storage.dledger.protocol.PullEntriesRequest; 24 | import io.openmessaging.storage.dledger.protocol.PullEntriesResponse; 25 | import io.openmessaging.storage.dledger.protocol.PushEntryRequest; 26 | import io.openmessaging.storage.dledger.protocol.PushEntryResponse; 27 | import io.openmessaging.storage.dledger.protocol.VoteRequest; 28 | import io.openmessaging.storage.dledger.protocol.VoteResponse; 29 | 30 | import java.util.concurrent.CompletableFuture; 31 | 32 | public interface DLedgerRaftProtocolHandler { 33 | 34 | CompletableFuture handleVote(VoteRequest request) throws Exception; 35 | 36 | CompletableFuture handleHeartBeat(HeartBeatRequest request) throws Exception; 37 | 38 | CompletableFuture handlePull(PullEntriesRequest request) throws Exception; 39 | 40 | CompletableFuture handlePush(PushEntryRequest request) throws Exception; 41 | 42 | CompletableFuture handleInstallSnapshot(InstallSnapshotRequest request) throws Exception; 43 | 44 | } 45 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/handler/DLedgerRpcProtocolHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.handler; 18 | 19 | public interface DLedgerRpcProtocolHandler extends DLedgerClientProtocolHandler, DLedgerRaftProtocolHandler { 20 | 21 | } 22 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerClientProtocol.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.protocol; 18 | 19 | import io.openmessaging.storage.dledger.protocol.AppendEntryRequest; 20 | import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; 21 | import io.openmessaging.storage.dledger.protocol.GetEntriesRequest; 22 | import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; 23 | import io.openmessaging.storage.dledger.protocol.LeadershipTransferRequest; 24 | import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; 25 | import io.openmessaging.storage.dledger.protocol.MetadataRequest; 26 | import io.openmessaging.storage.dledger.protocol.MetadataResponse; 27 | 28 | import java.util.concurrent.CompletableFuture; 29 | 30 | /** 31 | * Both the RaftLogServer(inbound) and RaftRpcService (outbound) should implement this protocol 32 | */ 33 | public interface DLedgerClientProtocol { 34 | 35 | CompletableFuture get(GetEntriesRequest request) throws Exception; 36 | 37 | CompletableFuture append(AppendEntryRequest request) throws Exception; 38 | 39 | CompletableFuture metadata(MetadataRequest request) throws Exception; 40 | 41 | CompletableFuture leadershipTransfer(LeadershipTransferRequest request) throws Exception; 42 | } 43 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerProtocol.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.protocol; 18 | 19 | 20 | /** 21 | * Both the RaftLogServer(inbound) and RaftRpcService (outbound) should implement this protocol 22 | */ 23 | public interface DLedgerProtocol extends DLedgerClientProtocol, DLedgerRaftProtocol { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/protocol/DLedgerRaftProtocol.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.protocol; 18 | 19 | import io.openmessaging.storage.dledger.protocol.HeartBeatRequest; 20 | import io.openmessaging.storage.dledger.protocol.HeartBeatResponse; 21 | import io.openmessaging.storage.dledger.protocol.InstallSnapshotRequest; 22 | import io.openmessaging.storage.dledger.protocol.InstallSnapshotResponse; 23 | import io.openmessaging.storage.dledger.protocol.PullEntriesRequest; 24 | import io.openmessaging.storage.dledger.protocol.PullEntriesResponse; 25 | import io.openmessaging.storage.dledger.protocol.PushEntryRequest; 26 | import io.openmessaging.storage.dledger.protocol.PushEntryResponse; 27 | import io.openmessaging.storage.dledger.protocol.VoteRequest; 28 | import io.openmessaging.storage.dledger.protocol.VoteResponse; 29 | 30 | import java.util.concurrent.CompletableFuture; 31 | 32 | public interface DLedgerRaftProtocol { 33 | 34 | CompletableFuture vote(VoteRequest request) throws Exception; 35 | 36 | CompletableFuture heartBeat(HeartBeatRequest request) throws Exception; 37 | 38 | CompletableFuture pull(PullEntriesRequest request) throws Exception; 39 | 40 | CompletableFuture push(PushEntryRequest request) throws Exception; 41 | 42 | CompletableFuture installSnapshot(InstallSnapshotRequest request) throws Exception; 43 | 44 | } 45 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/userdefine/UserDefineCommandHeader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.userdefine; 18 | 19 | import org.apache.rocketmq.remoting.CommandCustomHeader; 20 | import org.apache.rocketmq.remoting.exception.RemotingCommandException; 21 | 22 | 23 | public class UserDefineCommandHeader implements CommandCustomHeader { 24 | 25 | private Integer requestTypeCode; 26 | 27 | public UserDefineCommandHeader() { 28 | 29 | } 30 | 31 | public UserDefineCommandHeader(int requestTypeCode) { 32 | this.requestTypeCode = requestTypeCode; 33 | } 34 | 35 | @Override 36 | public void checkFields() throws RemotingCommandException { 37 | 38 | } 39 | 40 | public void setRequestTypeCode(Integer requestTypeCode) { 41 | this.requestTypeCode = requestTypeCode; 42 | } 43 | 44 | public Integer getRequestTypeCode() { 45 | return requestTypeCode; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/userdefine/UserDefineProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.userdefine; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerServer; 20 | import java.lang.reflect.ParameterizedType; 21 | import java.lang.reflect.Type; 22 | import java.util.concurrent.CompletableFuture; 23 | 24 | public abstract class UserDefineProcessor { 25 | 26 | protected final DLedgerServer dLedgerServer; 27 | 28 | public UserDefineProcessor(DLedgerServer dLedgerServer) { 29 | this.dLedgerServer = dLedgerServer; 30 | } 31 | 32 | public abstract CompletableFuture handleRequest(T t); 33 | 34 | public abstract Integer getRequestTypeCode(); 35 | 36 | public Type getRequestType() { 37 | ParameterizedType parameterizedType = (ParameterizedType) this.getClass().getGenericSuperclass(); 38 | return parameterizedType.getActualTypeArguments()[0]; 39 | } 40 | 41 | 42 | public Type getResponseType() { 43 | ParameterizedType parameterizedType = (ParameterizedType) this.getClass().getGenericSuperclass(); 44 | return parameterizedType.getActualTypeArguments()[1]; 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/userdefine/UserDefineRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.userdefine; 18 | 19 | import io.openmessaging.storage.dledger.protocol.RequestOrResponse; 20 | 21 | public abstract class UserDefineRequest extends RequestOrResponse { 22 | 23 | public abstract int getRequestTypeCode(); 24 | 25 | } 26 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/protocol/userdefine/UserDefineResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.protocol.userdefine; 18 | 19 | import io.openmessaging.storage.dledger.protocol.RequestOrResponse; 20 | 21 | public class UserDefineResponse extends RequestOrResponse { 22 | } 23 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/DownloadSnapshot.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | public class DownloadSnapshot { 20 | 21 | private SnapshotMeta meta; 22 | 23 | private byte[] data; 24 | 25 | public DownloadSnapshot() { 26 | } 27 | 28 | public DownloadSnapshot(SnapshotMeta meta, byte[] data) { 29 | this.meta = meta; 30 | this.data = data; 31 | } 32 | 33 | public SnapshotMeta getMeta() { 34 | return meta; 35 | } 36 | 37 | public void setMeta(SnapshotMeta meta) { 38 | this.meta = meta; 39 | } 40 | 41 | public byte[] getData() { 42 | return data; 43 | } 44 | 45 | public void setData(byte[] data) { 46 | this.data = data; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotEntryResetStrategy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerConfig; 20 | 21 | public enum SnapshotEntryResetStrategy { 22 | /** 23 | * Delete all entries in (..., lastIncludedIndex] in sync mode 24 | */ 25 | RESET_ALL_SYNC, 26 | /** 27 | * Delete all entries in (..., lastIncludedIndex] in async mode 28 | */ 29 | RESET_ALL_ASYNC, 30 | 31 | /** 32 | * Delete all entries in (..., lastIncludedIndex] in the configured time {@link DLedgerConfig#getResetSnapshotEntriesDelayTime()} 33 | */ 34 | RESET_ALL_LATER, 35 | 36 | /** 37 | * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in sync mode 38 | */ 39 | RESET_BUT_KEEP_SOME_SYNC, 40 | 41 | /** 42 | * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in async mode 43 | */ 44 | RESET_BUT_KEEP_SOME_ASYNC, 45 | 46 | /** 47 | * Delete all entries in (..., lastIncludedIndex - keepEntriesNum {@link DLedgerConfig#getResetSnapshotEntriesButKeepLastEntriesNum()}] in the configured time {@link DLedgerConfig#getResetSnapshotEntriesDelayTime()} 48 | */ 49 | RESET_BUT_KEEP_SOME_LATER, 50 | } -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotMeta.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | public class SnapshotMeta { 20 | 21 | private long lastIncludedIndex; 22 | private long lastIncludedTerm; 23 | 24 | public SnapshotMeta(long lastIncludedIndex, long lastIncludedTerm) { 25 | this.lastIncludedIndex = lastIncludedIndex; 26 | this.lastIncludedTerm = lastIncludedTerm; 27 | } 28 | 29 | public long getLastIncludedIndex() { 30 | return lastIncludedIndex; 31 | } 32 | 33 | public void setLastIncludedIndex(int lastIncludedIndex) { 34 | this.lastIncludedIndex = lastIncludedIndex; 35 | } 36 | 37 | public long getLastIncludedTerm() { 38 | return lastIncludedTerm; 39 | } 40 | 41 | public void setLastIncludedTerm(int lastIncludedTerm) { 42 | this.lastIncludedTerm = lastIncludedTerm; 43 | } 44 | 45 | @Override 46 | public String toString() { 47 | return "SnapshotMeta{" + 48 | "lastIncludedIndex=" + lastIncludedIndex + 49 | ", lastIncludedTerm=" + lastIncludedTerm + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | import java.io.IOException; 20 | 21 | /** 22 | * Reader for snapshot 23 | */ 24 | public interface SnapshotReader { 25 | 26 | SnapshotMeta load() throws IOException; 27 | 28 | SnapshotMeta getSnapshotMeta(); 29 | 30 | String getSnapshotStorePath(); 31 | 32 | DownloadSnapshot generateDownloadSnapshot(); 33 | } 34 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | import java.util.HashMap; 20 | import java.util.Map; 21 | 22 | public enum SnapshotStatus { 23 | 24 | UNKNOWN(-1), 25 | SUCCESS(0), 26 | FAIL(10001), 27 | EXPIRED(10002); 28 | 29 | private static Map codeMap = new HashMap<>(); 30 | 31 | static { 32 | for (SnapshotStatus status : SnapshotStatus.values()) { 33 | codeMap.put(status.code, status); 34 | } 35 | } 36 | 37 | private int code; 38 | 39 | SnapshotStatus(int code) { 40 | this.code = code; 41 | } 42 | 43 | public static SnapshotStatus valueOf(int code) { 44 | SnapshotStatus tmp = codeMap.get(code); 45 | if (tmp != null) { 46 | return tmp; 47 | } else { 48 | return UNKNOWN; 49 | } 50 | } 51 | 52 | public int getCode() { 53 | return code; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | public interface SnapshotStore { 20 | 21 | SnapshotWriter createSnapshotWriter(); 22 | 23 | SnapshotReader createSnapshotReader(); 24 | 25 | boolean downloadSnapshot(DownloadSnapshot downloadSnapshot); 26 | 27 | void deleteExpiredSnapshot(long maxReservedSnapshotNum); 28 | 29 | long getSnapshotNum(); 30 | } 31 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/SnapshotWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot; 18 | 19 | import java.io.IOException; 20 | 21 | /** 22 | * Writer for snapshot 23 | */ 24 | public interface SnapshotWriter { 25 | 26 | void save(SnapshotStatus status) throws IOException; 27 | 28 | void setSnapshotMeta(SnapshotMeta snapshotMeta); 29 | 30 | String getSnapshotStorePath(); 31 | } 32 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/LoadSnapshotHook.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot.hook; 18 | 19 | import io.openmessaging.storage.dledger.snapshot.SnapshotMeta; 20 | import io.openmessaging.storage.dledger.snapshot.SnapshotReader; 21 | 22 | public interface LoadSnapshotHook extends SnapshotHook { 23 | 24 | void registerSnapshotMeta(SnapshotMeta snapshotMeta); 25 | 26 | SnapshotReader getSnapshotReader(); 27 | } 28 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SaveSnapshotHook.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot.hook; 18 | 19 | import io.openmessaging.storage.dledger.snapshot.SnapshotMeta; 20 | import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; 21 | 22 | public interface SaveSnapshotHook extends SnapshotHook { 23 | 24 | void registerSnapshotMeta(final SnapshotMeta meta); 25 | 26 | SnapshotWriter getSnapshotWriter(); 27 | } 28 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/snapshot/hook/SnapshotHook.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.snapshot.hook; 18 | 19 | import io.openmessaging.storage.dledger.snapshot.SnapshotStatus; 20 | 21 | public interface SnapshotHook { 22 | 23 | void doCallBack(SnapshotStatus status); 24 | } 25 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/ApplyEntry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine; 18 | 19 | import io.openmessaging.storage.dledger.entry.DLedgerEntry; 20 | 21 | public class ApplyEntry { 22 | 23 | private DLedgerEntry entry; 24 | 25 | private T resp; 26 | 27 | public ApplyEntry(DLedgerEntry entry) { 28 | this.entry = entry; 29 | } 30 | 31 | public void setResp(T resp) { 32 | this.resp = resp; 33 | } 34 | 35 | public T getResp() { 36 | return resp; 37 | } 38 | 39 | public void setEntry(DLedgerEntry entry) { 40 | this.entry = entry; 41 | } 42 | 43 | public DLedgerEntry getEntry() { 44 | return entry; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/NoOpStatemachine.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine; 18 | 19 | import io.openmessaging.storage.dledger.exception.DLedgerException; 20 | import io.openmessaging.storage.dledger.snapshot.SnapshotReader; 21 | import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; 22 | 23 | public class NoOpStatemachine implements StateMachine { 24 | @Override 25 | public void onApply(ApplyEntryIterator iter) { 26 | while (iter.hasNext()) { 27 | iter.next(); 28 | } 29 | } 30 | 31 | @Override 32 | public boolean onSnapshotSave(SnapshotWriter writer) { 33 | return false; 34 | } 35 | 36 | @Override 37 | public boolean onSnapshotLoad(SnapshotReader reader) { 38 | return false; 39 | } 40 | 41 | @Override 42 | public void onShutdown() { 43 | 44 | } 45 | 46 | @Override 47 | public void onError(DLedgerException error) { 48 | 49 | } 50 | 51 | @Override 52 | public String getBindDLedgerId() { 53 | return null; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/statemachine/StateMachine.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine; 18 | 19 | import io.openmessaging.storage.dledger.exception.DLedgerException; 20 | import io.openmessaging.storage.dledger.snapshot.SnapshotReader; 21 | import io.openmessaging.storage.dledger.snapshot.SnapshotWriter; 22 | import io.openmessaging.storage.dledger.utils.DLedgerUtils; 23 | 24 | /** 25 | * Finite state machine, which should be implemented by user. 26 | */ 27 | public interface StateMachine { 28 | 29 | /** 30 | * Update the user statemachine with a batch a tasks that can be accessed 31 | * through |iterator|. 32 | * 33 | * @param iter iterator of committed entry 34 | */ 35 | void onApply(final ApplyEntryIterator iter); 36 | 37 | /** 38 | * User defined snapshot generate function. 39 | * 40 | * @param writer snapshot writer 41 | * @return true on success 42 | */ 43 | boolean onSnapshotSave(final SnapshotWriter writer); 44 | 45 | /** 46 | * User defined snapshot load function. 47 | * 48 | * @param reader snapshot reader 49 | * @return true on success 50 | */ 51 | boolean onSnapshotLoad(final SnapshotReader reader); 52 | 53 | /** 54 | * Invoked once when the raft node was shut down. 55 | * Default do nothing 56 | */ 57 | void onShutdown(); 58 | 59 | /** 60 | * Once a critical error occurs, disallow any task enter the Dledger node 61 | * until the error has been fixed and restart it. 62 | * 63 | * @param error DLedger error message 64 | */ 65 | void onError(final DLedgerException error); 66 | 67 | /** 68 | * User must create DLedgerId by this method, it will generate the DLedgerId with format like that: 'dLedgerGroupId#dLedgerSelfId' 69 | * @param dLedgerGroupId the group id of the DLedgerServer 70 | * @param dLedgerSelfId the self id of the DLedgerServer 71 | * @return generated unique DLedgerId 72 | */ 73 | default String generateDLedgerId(String dLedgerGroupId, String dLedgerSelfId) { 74 | return DLedgerUtils.generateDLedgerId(dLedgerGroupId, dLedgerSelfId); 75 | } 76 | 77 | /** 78 | * User should return the DLedgerId which can be created by the method 'StateMachine#generateDLedgerId' 79 | * @return DLedgerId 80 | */ 81 | String getBindDLedgerId(); 82 | } 83 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/store/DLedgerStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.store; 18 | 19 | import io.openmessaging.storage.dledger.MemberState; 20 | import io.openmessaging.storage.dledger.entry.DLedgerEntry; 21 | 22 | public abstract class DLedgerStore { 23 | 24 | public abstract MemberState getMemberState(); 25 | 26 | public abstract DLedgerEntry appendAsLeader(DLedgerEntry entry); 27 | 28 | public abstract DLedgerEntry appendAsFollower(DLedgerEntry entry, long leaderTerm, String leaderId); 29 | 30 | public abstract DLedgerEntry get(Long index); 31 | 32 | public abstract long getLedgerEndTerm(); 33 | 34 | public abstract long getLedgerEndIndex(); 35 | 36 | public abstract long getLedgerBeforeBeginIndex(); 37 | 38 | public abstract long getLedgerBeforeBeginTerm(); 39 | 40 | protected void updateLedgerEndIndexAndTerm() { 41 | if (getMemberState() != null) { 42 | getMemberState().updateLedgerIndexAndTerm(getLedgerEndIndex(), getLedgerEndTerm()); 43 | } 44 | } 45 | 46 | public abstract void flush(); 47 | 48 | public long truncate(DLedgerEntry entry, long leaderTerm, String leaderId) { 49 | return -1; 50 | } 51 | 52 | /** 53 | * truncate all entries in [truncateIndex ..] 54 | * @param truncateIndex truncate process since where 55 | * @return after truncate, store's end index 56 | */ 57 | public abstract long truncate(long truncateIndex); 58 | 59 | /** 60 | * reset store's first entry, clear all entries in [.. beforeBeginIndex], make beforeBeginIndex + 1 to be first entry's index 61 | * @param beforeBeginIndex after reset process, beforeBegin entry's index 62 | * @param beforeBeginTerm after reset process, beforeBegin entry's term 63 | * @return after reset, store's first log index 64 | */ 65 | public abstract long reset(long beforeBeginIndex, long beforeBeginTerm); 66 | 67 | public abstract void resetOffsetAfterSnapshot(DLedgerEntry entry); 68 | 69 | public abstract void updateIndexAfterLoadingSnapshot(long lastIncludedIndex, long lastIncludedTerm); 70 | 71 | public abstract DLedgerEntry getFirstLogOfTargetTerm(long targetTerm, long endIndex); 72 | 73 | public abstract void startup(); 74 | 75 | public abstract void shutdown(); 76 | 77 | } 78 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/store/file/ReferenceResource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.store.file; 18 | 19 | import java.util.concurrent.atomic.AtomicLong; 20 | 21 | public abstract class ReferenceResource { 22 | protected final AtomicLong refCount = new AtomicLong(1); 23 | protected volatile boolean available = true; 24 | protected volatile boolean cleanupOver = false; 25 | private volatile long firstShutdownTimestamp = 0; 26 | 27 | public synchronized boolean hold() { 28 | if (this.isAvailable()) { 29 | if (this.refCount.getAndIncrement() > 0) { 30 | return true; 31 | } else { 32 | this.refCount.getAndDecrement(); 33 | } 34 | } 35 | 36 | return false; 37 | } 38 | 39 | public boolean isAvailable() { 40 | return this.available; 41 | } 42 | 43 | public void shutdown(final long intervalForcibly) { 44 | if (this.available) { 45 | this.available = false; 46 | this.firstShutdownTimestamp = System.currentTimeMillis(); 47 | this.release(); 48 | } else if (this.getRefCount() > 0) { 49 | if ((System.currentTimeMillis() - this.firstShutdownTimestamp) >= intervalForcibly) { 50 | this.refCount.set(-1000 - this.getRefCount()); 51 | this.release(); 52 | } 53 | } 54 | } 55 | 56 | public void release() { 57 | long value = this.refCount.decrementAndGet(); 58 | if (value > 0) { 59 | return; 60 | } 61 | 62 | synchronized (this) { 63 | 64 | this.cleanupOver = this.cleanup(value); 65 | } 66 | } 67 | 68 | public long getRefCount() { 69 | return this.refCount.get(); 70 | } 71 | 72 | public abstract boolean cleanup(final long currentRef); 73 | 74 | public boolean isCleanupOver() { 75 | return this.refCount.get() <= 0 && this.cleanupOver; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/store/file/SelectMmapBufferResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.store.file; 18 | 19 | import java.nio.ByteBuffer; 20 | 21 | public class SelectMmapBufferResult { 22 | 23 | private final long startOffset; 24 | 25 | private final ByteBuffer byteBuffer; 26 | protected MmapFile mappedFile; 27 | private int size; 28 | 29 | public SelectMmapBufferResult(long startOffset, ByteBuffer byteBuffer, int size, MmapFile mappedFile) { 30 | this.startOffset = startOffset; 31 | this.byteBuffer = byteBuffer; 32 | this.size = size; 33 | this.mappedFile = mappedFile; 34 | } 35 | 36 | public ByteBuffer getByteBuffer() { 37 | return byteBuffer; 38 | } 39 | 40 | public int getSize() { 41 | return size; 42 | } 43 | 44 | public void setSize(final int s) { 45 | this.size = s; 46 | this.byteBuffer.limit(this.size); 47 | } 48 | 49 | public MmapFile getMappedFile() { 50 | return mappedFile; 51 | } 52 | 53 | public synchronized void release() { 54 | if (this.mappedFile != null) { 55 | this.mappedFile.release(); 56 | this.mappedFile = null; 57 | } 58 | } 59 | 60 | public long getStartOffset() { 61 | return startOffset; 62 | } 63 | 64 | public static void release(SelectMmapBufferResult sbr) { 65 | if (sbr != null) { 66 | sbr.release(); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/utils/BytesUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.utils; 18 | 19 | public class BytesUtil { 20 | 21 | public static byte[] intToBytes(int value) { 22 | byte[] src = new byte[4]; 23 | src[3] = (byte) ((value >> 24) & 0xFF); 24 | src[2] = (byte) ((value >> 16) & 0xFF); 25 | src[1] = (byte) ((value >> 8) & 0xFF); 26 | src[0] = (byte) (value & 0xFF); 27 | return src; 28 | } 29 | 30 | public static int bytesToInt(byte[] src, int offset) { 31 | int value; 32 | value = (int) ((src[offset] & 0xFF) 33 | | ((src[offset + 1] & 0xFF) << 8) 34 | | ((src[offset + 2] & 0xFF) << 16) 35 | | ((src[offset + 3] & 0xFF) << 24)); 36 | return value; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/utils/Pair.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.utils; 18 | 19 | public class Pair { 20 | 21 | private K k; 22 | private V v; 23 | 24 | public Pair(K k, V v) { 25 | this.k = k; 26 | this.v = v; 27 | } 28 | 29 | public K getKey() { 30 | return k; 31 | } 32 | 33 | public V getValue() { 34 | return v; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/utils/PreConditions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.utils; 18 | 19 | import io.openmessaging.storage.dledger.exception.DLedgerException; 20 | import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; 21 | 22 | public class PreConditions { 23 | 24 | public static void check(boolean expression, DLedgerResponseCode code) throws DLedgerException { 25 | check(expression, code, null); 26 | } 27 | 28 | public static void check(boolean expression, DLedgerResponseCode code, String message) throws DLedgerException { 29 | if (!expression) { 30 | message = message == null ? code.toString() 31 | : code.toString() + " " + message; 32 | throw new DLedgerException(code, message); 33 | } 34 | } 35 | 36 | public static void check(boolean expression, DLedgerResponseCode code, String format, 37 | Object... args) throws DLedgerException { 38 | if (!expression) { 39 | String message = code.toString() + " " + String.format(format, args); 40 | throw new DLedgerException(code, message); 41 | } 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /dledger/src/main/java/io/openmessaging/storage/dledger/utils/Quota.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.utils; 18 | 19 | public class Quota { 20 | 21 | private final int max; 22 | 23 | private final int[] samples; 24 | private final long[] timeVec; 25 | 26 | private final int window; 27 | 28 | public Quota(int max) { 29 | this(5, max); 30 | } 31 | public Quota(int window, int max) { 32 | if (window < 5) { 33 | window = 5; 34 | } 35 | this.max = max; 36 | this.window = window; 37 | this.samples = new int[window]; 38 | this.timeVec = new long[window]; 39 | } 40 | 41 | private int index(long currTimeMs) { 42 | return (int) (second(currTimeMs) % window); 43 | } 44 | 45 | private long second(long currTimeMs) { 46 | return currTimeMs / 1000; 47 | } 48 | 49 | public void sample(int value) { 50 | long timeMs = System.currentTimeMillis(); 51 | int index = index(timeMs); 52 | long second = second(timeMs); 53 | if (timeVec[index] != second) { 54 | timeVec[index] = second; 55 | samples[index] = value; 56 | } else { 57 | samples[index] += value; 58 | } 59 | 60 | } 61 | 62 | public boolean validateNow() { 63 | long timeMs = System.currentTimeMillis(); 64 | int index = index(timeMs); 65 | long second = second(timeMs); 66 | if (timeVec[index] == second) { 67 | return samples[index] >= max; 68 | } 69 | return false; 70 | } 71 | 72 | public int leftNow() { 73 | long timeMs = System.currentTimeMillis(); 74 | return (int) ((second(timeMs) + 1) * 1000 - timeMs); 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /dledger/src/main/resources/config.example.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10001;n2-127.0.0.1:10002 5 | preferredLeaderIds: n0 6 | - group: g1 7 | selfId: a0 8 | peers: a0-127.0.0.1:10000;a1-127.0.0.1:10001;a2-127.0.0.1:10002 9 | preferredLeaderIds: a1 10 | - group: g2 11 | selfId: b0 12 | peers: b0-127.0.0.1:10000;b1-127.0.0.1:10001;b2-127.0.0.1:10002 13 | preferredLeaderIds: b2 14 | -------------------------------------------------------------------------------- /dledger/src/main/resources/config.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10000;n2-127.0.0.1:10000 5 | preferredLeaderIds: n0 6 | - group: g0 7 | selfId: n1 8 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10000;n2-127.0.0.1:10000 9 | preferredLeaderIds: n0 10 | - group: g0 11 | selfId: n2 12 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10000;n2-127.0.0.1:10000 13 | preferredLeaderIds: n0 14 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/ServerTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger; 18 | 19 | import io.openmessaging.storage.dledger.util.FileTestUtil; 20 | import org.junit.jupiter.api.AfterEach; 21 | 22 | import java.util.ArrayList; 23 | import java.util.List; 24 | import java.util.Random; 25 | import java.util.concurrent.atomic.AtomicInteger; 26 | 27 | public class ServerTestBase { 28 | 29 | private static final AtomicInteger PORT_COUNTER = new AtomicInteger(30000); 30 | private static Random random = new Random(); 31 | protected List bases = new ArrayList<>(); 32 | 33 | public static int nextPort() { 34 | return PORT_COUNTER.addAndGet(10 + random.nextInt(10)); 35 | } 36 | 37 | @AfterEach 38 | public synchronized void shutdown() { 39 | for (String base : bases) { 40 | try { 41 | FileTestUtil.deleteFile(base); 42 | } catch (Throwable ignored) { 43 | 44 | } 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/common/NamedThreadFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.common; 18 | 19 | import io.openmessaging.storage.dledger.common.NamedThreadFactory; 20 | import java.util.concurrent.atomic.AtomicInteger; 21 | import org.junit.jupiter.api.Assertions; 22 | import org.junit.jupiter.api.Test; 23 | 24 | public class NamedThreadFactoryTest { 25 | 26 | @Test 27 | public void testNamedThreadFactory() { 28 | 29 | NamedThreadFactory threadFactory = new NamedThreadFactory("DledgerThread"); 30 | Runnable runnable = () -> { 31 | }; 32 | Thread thread = threadFactory.newThread(runnable); 33 | Assertions.assertEquals("DledgerThread-1", thread.getName()); 34 | Assertions.assertFalse(thread.isDaemon()); 35 | 36 | threadFactory = new NamedThreadFactory("DledgerThread", true); 37 | Thread thread1 = threadFactory.newThread(runnable); 38 | Assertions.assertEquals("DledgerThread-1", thread1.getName()); 39 | Assertions.assertTrue(thread1.isDaemon()); 40 | 41 | threadFactory = new NamedThreadFactory(null, "DledgerThread", true); 42 | Thread thread2 = threadFactory.newThread(runnable); 43 | Assertions.assertEquals("DledgerThread", thread2.getName()); 44 | Assertions.assertTrue(thread2.isDaemon()); 45 | 46 | threadFactory = new NamedThreadFactory(new AtomicInteger(0), "DledgerThread", true); 47 | Thread thread3 = threadFactory.newThread(runnable); 48 | Assertions.assertEquals("DledgerThread-1", thread3.getName()); 49 | Assertions.assertTrue(thread2.isDaemon()); 50 | 51 | } 52 | 53 | } -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotReaderTest.java: -------------------------------------------------------------------------------- 1 | package io.openmessaging.storage.dledger.snapshot; 2 | 3 | import com.alibaba.fastjson.JSON; 4 | import io.openmessaging.storage.dledger.snapshot.file.FileSnapshotReader; 5 | import io.openmessaging.storage.dledger.util.FileTestUtil; 6 | import io.openmessaging.storage.dledger.utils.IOUtils; 7 | import org.junit.jupiter.api.Assertions; 8 | import org.junit.jupiter.api.Test; 9 | 10 | import java.io.File; 11 | import java.io.IOException; 12 | import java.util.UUID; 13 | 14 | public class SnapshotReaderTest { 15 | 16 | @Test 17 | public void testReaderLoad() throws IOException { 18 | String metaFilePath = FileTestUtil.TEST_BASE + File.separator + SnapshotManager.SNAPSHOT_META_FILE; 19 | try { 20 | SnapshotMeta snapshotMeta = new SnapshotMeta(10, 0); 21 | IOUtils.string2File(JSON.toJSONString(snapshotMeta), metaFilePath); 22 | 23 | SnapshotReader reader = new FileSnapshotReader(FileTestUtil.TEST_BASE); 24 | Assertions.assertNull(reader.getSnapshotMeta()); 25 | Assertions.assertEquals(reader.getSnapshotStorePath(), FileTestUtil.TEST_BASE); 26 | Assertions.assertEquals(reader.load().toString(), snapshotMeta.toString()); 27 | } finally { 28 | IOUtils.deleteFile(new File(metaFilePath)); 29 | } 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotStoreTest.java: -------------------------------------------------------------------------------- 1 | package io.openmessaging.storage.dledger.snapshot; 2 | 3 | import com.alibaba.fastjson.JSON; 4 | import io.openmessaging.storage.dledger.snapshot.file.FileSnapshotStore; 5 | import io.openmessaging.storage.dledger.util.FileTestUtil; 6 | import io.openmessaging.storage.dledger.utils.IOUtils; 7 | import org.junit.jupiter.api.Assertions; 8 | import org.junit.jupiter.api.Test; 9 | 10 | import java.io.File; 11 | import java.io.IOException; 12 | 13 | public class SnapshotStoreTest { 14 | 15 | @Test 16 | public void testCreateReaderAndWriter() throws IOException { 17 | final long lastSnapshotIndex = 10; 18 | String baseDir = null; 19 | try { 20 | baseDir = FileTestUtil.createTestDir("SnapshotStoreTest"); 21 | FileSnapshotStore writerStore = new FileSnapshotStore(baseDir); 22 | SnapshotWriter writer = writerStore.createSnapshotWriter(); 23 | Assertions.assertNotNull(writer); 24 | SnapshotMeta writerMeta = new SnapshotMeta(lastSnapshotIndex, 0); 25 | writer.setSnapshotMeta(writerMeta); 26 | writer.save(SnapshotStatus.SUCCESS); 27 | 28 | FileSnapshotStore readerStore = new FileSnapshotStore(baseDir); 29 | SnapshotReader reader = readerStore.createSnapshotReader(); 30 | Assertions.assertNotNull(reader); 31 | SnapshotMeta readerMeta = reader.load(); 32 | Assertions.assertEquals(writerMeta.toString(), readerMeta.toString()); 33 | } finally { 34 | IOUtils.deleteFile(new File(baseDir + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex)); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/snapshot/SnapshotWriterTest.java: -------------------------------------------------------------------------------- 1 | package io.openmessaging.storage.dledger.snapshot; 2 | 3 | import com.alibaba.fastjson.JSON; 4 | import io.openmessaging.storage.dledger.snapshot.file.FileSnapshotStore; 5 | import io.openmessaging.storage.dledger.snapshot.file.FileSnapshotWriter; 6 | import io.openmessaging.storage.dledger.util.FileTestUtil; 7 | import io.openmessaging.storage.dledger.utils.IOUtils; 8 | import org.junit.jupiter.api.Assertions; 9 | import org.junit.jupiter.api.Test; 10 | 11 | import java.io.File; 12 | import java.io.IOException; 13 | 14 | public class SnapshotWriterTest { 15 | 16 | @Test 17 | public void testWriterSave() throws IOException { 18 | final long lastSnapshotIndex = 10; 19 | SnapshotWriter writer = new FileSnapshotWriter(FileTestUtil.TEST_BASE + File.separator + "tmp", new FileSnapshotStore(FileTestUtil.TEST_BASE)); 20 | 21 | SnapshotMeta snapshotMeta = new SnapshotMeta(lastSnapshotIndex, 0); 22 | writer.setSnapshotMeta(snapshotMeta); 23 | writer.save(SnapshotStatus.SUCCESS); 24 | 25 | String testDir = FileTestUtil.TEST_BASE + File.separator + SnapshotManager.SNAPSHOT_DIR_PREFIX + lastSnapshotIndex; 26 | try { 27 | Assertions.assertEquals(IOUtils.file2String(testDir + File.separator + 28 | SnapshotManager.SNAPSHOT_META_FILE), JSON.toJSONString(snapshotMeta)); 29 | } finally { 30 | IOUtils.deleteFile(new File(testDir)); 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/MockSnapshotFile.java: -------------------------------------------------------------------------------- 1 | package io.openmessaging.storage.dledger.statemachine; 2 | 3 | import io.openmessaging.storage.dledger.utils.IOUtils; 4 | import org.slf4j.Logger; 5 | import org.slf4j.LoggerFactory; 6 | 7 | import java.io.File; 8 | import java.io.IOException; 9 | 10 | public class MockSnapshotFile { 11 | 12 | private static final Logger logger = LoggerFactory.getLogger(MockSnapshotFile.class); 13 | 14 | private final String snapshotStorePath; 15 | 16 | public MockSnapshotFile(String snapshotStorePath) { 17 | this.snapshotStorePath = snapshotStorePath; 18 | } 19 | 20 | public boolean save(final long value) { 21 | try { 22 | IOUtils.string2File(String.valueOf(value), snapshotStorePath); 23 | return true; 24 | } catch (IOException e) { 25 | logger.error("Unable to save snapshot data", e); 26 | return false; 27 | } 28 | } 29 | 30 | public long load() throws IOException { 31 | String str = IOUtils.file2String(new File(snapshotStorePath)); 32 | if (str != null && !str.isEmpty()) { 33 | return Long.parseLong(str); 34 | } else { 35 | throw new IOException("Unable to load snapshot data from " + snapshotStorePath); 36 | } 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/RegisterSnapshotFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2020 the original author or authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine; 18 | 19 | import io.openmessaging.storage.dledger.utils.IOUtils; 20 | import org.slf4j.Logger; 21 | import org.slf4j.LoggerFactory; 22 | 23 | import java.io.IOException; 24 | import java.util.Map; 25 | 26 | public class RegisterSnapshotFile { 27 | 28 | private static final Logger logger = LoggerFactory.getLogger(RegisterSnapshotFile.class); 29 | 30 | private final String snapshotStorePath; 31 | 32 | public RegisterSnapshotFile(String snapshotStorePath) { 33 | this.snapshotStorePath = snapshotStorePath; 34 | } 35 | 36 | public boolean save(final Map register) { 37 | try { 38 | IOUtils.object2File(snapshotStorePath, register); 39 | } catch (IOException e) { 40 | logger.error("Unable to save snapshot data", e); 41 | return false; 42 | } 43 | return true; 44 | } 45 | 46 | public Map load() throws IOException, ClassNotFoundException { 47 | Object object = IOUtils.file2Object(this.snapshotStorePath); 48 | if (object instanceof Map) { 49 | Map map = (Map) object; 50 | return map; 51 | } else { 52 | throw new IOException("Unable to load data from file to map"); 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine.register; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerServer; 20 | import io.openmessaging.storage.dledger.common.ReadClosure; 21 | import io.openmessaging.storage.dledger.common.ReadMode; 22 | import io.openmessaging.storage.dledger.common.Status; 23 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; 24 | 25 | import java.util.concurrent.CompletableFuture; 26 | 27 | public class RegisterReadProcessor extends UserDefineProcessor { 28 | 29 | private Integer requestTypeCode; 30 | 31 | public RegisterReadProcessor(DLedgerServer server) { 32 | super(server); 33 | RegisterReadRequest registerReadRequest = new RegisterReadRequest(0); 34 | this.requestTypeCode = registerReadRequest.getRequestTypeCode(); 35 | } 36 | @Override 37 | public CompletableFuture handleRequest(RegisterReadRequest registerReadRequest) { 38 | Integer key = registerReadRequest.getKey(); 39 | RegisterReadResponse response = new RegisterReadResponse(); 40 | response.setKey(key); 41 | CompletableFuture future = new CompletableFuture<>(); 42 | ReadClosure closure = new ReadClosure() { 43 | @Override 44 | public void done(Status status) { 45 | if (status.isOk()) { 46 | RegisterStateMachine registerStateMachine = (RegisterStateMachine) dLedgerServer.getStateMachine(); 47 | Integer value = registerStateMachine.getValue(key); 48 | response.setValue(value); 49 | future.complete(response); 50 | } else { 51 | response.setCode(status.code.getCode()); 52 | future.complete(response); 53 | } 54 | } 55 | }; 56 | dLedgerServer.handleRead(ReadMode.RAFT_LOG_READ, closure); 57 | return future; 58 | } 59 | 60 | @Override 61 | public Integer getRequestTypeCode() { 62 | return this.requestTypeCode; 63 | } 64 | 65 | } 66 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine.register; 18 | 19 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; 20 | 21 | public class RegisterReadRequest extends UserDefineRequest { 22 | 23 | private Integer key; 24 | 25 | public RegisterReadRequest(int key) { 26 | this.key = key; 27 | } 28 | 29 | public void setKey(Integer key) { 30 | this.key = key; 31 | } 32 | 33 | public Integer getKey() { 34 | return key; 35 | } 36 | 37 | @Override 38 | public int getRequestTypeCode() { 39 | return 1000; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/statemachine/register/RegisterReadResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.statemachine.register; 18 | 19 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineResponse; 20 | 21 | public class RegisterReadResponse extends UserDefineResponse { 22 | 23 | private Integer key; 24 | 25 | private Integer value; 26 | 27 | public void setKey(Integer key) { 28 | this.key = key; 29 | } 30 | 31 | public void setValue(Integer value) { 32 | this.value = value; 33 | } 34 | 35 | public Integer getKey() { 36 | return key; 37 | } 38 | 39 | public Integer getValue() { 40 | return value; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/store/file/DefaultMmapFileTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.store.file; 18 | 19 | import io.openmessaging.storage.dledger.ServerTestBase; 20 | import org.junit.jupiter.api.Assertions; 21 | import org.junit.jupiter.api.Test; 22 | 23 | public class DefaultMmapFileTest extends ServerTestBase { 24 | 25 | @Test 26 | public void testDefaultMmapFile() throws Exception{ 27 | 28 | String path = "/tmp/file/a/"; 29 | bases.add(path); 30 | DefaultMmapFile file = new DefaultMmapFile(path+"0000",1023); 31 | 32 | file.setStartPosition(1); 33 | file.setWrotePosition(2); 34 | file.setFlushedPosition(3); 35 | file.setCommittedPosition(1); 36 | 37 | Assertions.assertEquals(1, file.getStartPosition()); 38 | Assertions.assertEquals(2, file.getWrotePosition()); 39 | Assertions.assertEquals(3, file.getFlushedPosition()); 40 | Assertions.assertEquals(2, file.commit(1)); 41 | } 42 | 43 | 44 | } -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/util/FileTestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.util; 18 | 19 | import io.openmessaging.storage.dledger.utils.IOUtils; 20 | import java.io.File; 21 | import java.io.IOException; 22 | import java.util.UUID; 23 | 24 | public class FileTestUtil { 25 | 26 | public static final String TEST_BASE = File.separator + "tmp" + File.separator + "dledgerteststore"; 27 | 28 | public static String createTestDir() { 29 | return createTestDir(null); 30 | } 31 | 32 | public static String createTestDir(String prefix) { 33 | if (prefix == null) { 34 | prefix = "test"; 35 | } 36 | String baseDir = TEST_BASE + File.separator + prefix + "-" + UUID.randomUUID().toString(); 37 | final File file = new File(baseDir); 38 | if (file.exists()) { 39 | System.exit(1); 40 | } 41 | return baseDir; 42 | } 43 | 44 | public static void deleteFile(String fileName) throws IOException { 45 | IOUtils.deleteFile(new File(fileName)); 46 | } 47 | 48 | public static void deleteFile(File file) throws IOException { 49 | IOUtils.deleteFile(file); 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /dledger/src/test/java/io/openmessaging/storage/dledger/util/QuotaTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.util; 18 | 19 | import io.openmessaging.storage.dledger.utils.Quota; 20 | import org.junit.jupiter.api.Assertions; 21 | import org.junit.jupiter.api.Test; 22 | 23 | public class QuotaTest { 24 | 25 | 26 | // @Test 27 | // public void testValidateNow() throws Exception { 28 | // Quota quota = new Quota(100); 29 | // Thread.sleep(quota.leftNow() + 1); 30 | // for (int i = 0; i < 500; i++) { 31 | // if (i >= 100) { 32 | // Assertions.assertTrue(quota.validateNow()); 33 | // } else { 34 | // Assertions.assertFalse(quota.validateNow()); 35 | // } 36 | // quota.sample(1); 37 | // Assertions.assertTrue(quota.leftNow() < 1000 - i); 38 | // Assertions.assertTrue(quota.leftNow() > 1000 - i - 200); 39 | // Thread.sleep(1); 40 | // } 41 | // } 42 | } 43 | -------------------------------------------------------------------------------- /example/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | io.openmessaging.storage 8 | dledger-all 9 | 0.3.3-SNAPSHOT 10 | 11 | 12 | dledger-example 13 | 14 | 15 | 16 | io.openmessaging.storage 17 | dledger-proxy 18 | 19 | 20 | com.beust 21 | jcommander 22 | 23 | 24 | org.slf4j 25 | slf4j-simple 26 | 27 | 28 | io.dropwizard.metrics 29 | metrics-core 30 | 4.2.7 31 | 32 | 33 | org.junit.jupiter 34 | junit-jupiter-api 35 | test 36 | 37 | 38 | 39 | 40 | 41 | 42 | maven-assembly-plugin 43 | 44 | 45 | 46 | io.openmessaging.storage.dledger.example.CommandCli 47 | 48 | 49 | dledger-example 50 | false 51 | false 52 | 53 | jar-with-dependencies 54 | 55 | 56 | 57 | 58 | 59 | 60 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/appender/AppenderDLedger.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.appender; 18 | 19 | import com.alibaba.fastjson.JSON; 20 | import io.openmessaging.storage.dledger.DLedgerConfig; 21 | import io.openmessaging.storage.dledger.proxy.DLedgerProxy; 22 | import java.util.Collections; 23 | import java.util.List; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | public class AppenderDLedger { 28 | 29 | private static Logger logger = LoggerFactory.getLogger(AppenderDLedger.class); 30 | 31 | public static void bootstrapDLedger(List dLedgerConfigs) { 32 | if (dLedgerConfigs == null || dLedgerConfigs.isEmpty()) { 33 | logger.error("Bootstrap DLedger server error", new IllegalArgumentException("DLedgerConfigs is null or empty")); 34 | } 35 | DLedgerProxy dLedgerProxy = new DLedgerProxy(dLedgerConfigs); 36 | dLedgerProxy.startup(); 37 | logger.info("DLedgers start ok with config {}", JSON.toJSONString(dLedgerConfigs)); 38 | Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { 39 | private volatile boolean hasShutdown = false; 40 | 41 | @Override 42 | public void run() { 43 | synchronized (this) { 44 | logger.info("Shutdown hook was invoked"); 45 | if (!this.hasShutdown) { 46 | this.hasShutdown = true; 47 | long beginTime = System.currentTimeMillis(); 48 | dLedgerProxy.shutdown(); 49 | long consumingTimeTotal = System.currentTimeMillis() - beginTime; 50 | logger.info("Shutdown hook over, consuming total time(ms): {}", consumingTimeTotal); 51 | } 52 | } 53 | } 54 | }, "ShutdownHook")); 55 | } 56 | 57 | public static void bootstrapDLedger(DLedgerConfig dLedgerConfig) { 58 | bootstrapDLedger(Collections.singletonList(dLedgerConfig)); 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/appender/command/AppendCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.appender.command; 18 | 19 | import com.alibaba.fastjson.JSON; 20 | import com.beust.jcommander.Parameter; 21 | import com.beust.jcommander.Parameters; 22 | import io.openmessaging.storage.dledger.client.DLedgerClient; 23 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 24 | import io.openmessaging.storage.dledger.protocol.AppendEntryResponse; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | @Parameters(commandDescription = "Append data to AppenderDLedger") 29 | public class AppendCommand extends BaseCommand { 30 | 31 | private static Logger logger = LoggerFactory.getLogger(AppendCommand.class); 32 | 33 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 34 | private String group = "default"; 35 | 36 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 37 | private String peers = "n0-localhost:20911"; 38 | 39 | @Parameter(names = {"--data", "-d"}, description = "the data to append") 40 | private String data = "Hello"; 41 | 42 | @Parameter(names = {"--count", "-c"}, description = "append several times") 43 | private int count = 1; 44 | 45 | @Override 46 | public void doCommand() { 47 | DLedgerClient dLedgerClient = new DLedgerClient(group, peers); 48 | dLedgerClient.startup(); 49 | for (int i = 0; i < count; i++) { 50 | AppendEntryResponse response = dLedgerClient.append(data.getBytes()); 51 | logger.info("Append Result:{}", JSON.toJSONString(response)); 52 | } 53 | dLedgerClient.shutdown(); 54 | } 55 | } 56 | 57 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/appender/command/ConfigCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.appender.command; 18 | 19 | import com.beust.jcommander.Parameter; 20 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | 25 | public class ConfigCommand extends BaseCommand { 26 | 27 | private static Logger logger = LoggerFactory.getLogger(ConfigCommand.class); 28 | 29 | @Parameter(names = {"--config", "-c"}, description = "Config path of DLedger") 30 | private String config = "config.yaml"; 31 | 32 | @Override 33 | public void doCommand() { 34 | //doCommand 35 | } 36 | 37 | public String getConfigPath() { 38 | return config; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/appender/command/GetCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.appender.command; 18 | 19 | import com.alibaba.fastjson.JSON; 20 | import com.beust.jcommander.Parameter; 21 | import com.beust.jcommander.Parameters; 22 | import io.openmessaging.storage.dledger.client.DLedgerClient; 23 | import io.openmessaging.storage.dledger.entry.DLedgerEntry; 24 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 25 | import io.openmessaging.storage.dledger.protocol.GetEntriesResponse; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | @Parameters(commandDescription = "Get data from AppenderDLedger") 30 | public class GetCommand extends BaseCommand { 31 | 32 | private static Logger logger = LoggerFactory.getLogger(GetCommand.class); 33 | 34 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 35 | private String group = "default"; 36 | 37 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 38 | private String peers = "n0-localhost:20911"; 39 | 40 | @Parameter(names = {"--index", "-i"}, description = "get entry from index") 41 | private long index = 0; 42 | 43 | @Override 44 | public void doCommand() { 45 | DLedgerClient dLedgerClient = new DLedgerClient(group, peers); 46 | dLedgerClient.startup(); 47 | GetEntriesResponse response = dLedgerClient.get(index); 48 | logger.info("Get Result:{}", JSON.toJSONString(response)); 49 | if (response.getEntries() != null && response.getEntries().size() > 0) { 50 | for (DLedgerEntry entry : response.getEntries()) { 51 | logger.info("Get Result index:{} {}", entry.getIndex(), new String(entry.getBody())); 52 | } 53 | } 54 | dLedgerClient.shutdown(); 55 | } 56 | } -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/common/command/BaseCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.common.command; 18 | 19 | public abstract class BaseCommand { 20 | 21 | public abstract void doCommand(); 22 | } 23 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/common/command/LeadershipTransferCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.common.command; 18 | 19 | import com.alibaba.fastjson.JSON; 20 | import com.beust.jcommander.Parameter; 21 | import com.beust.jcommander.Parameters; 22 | import io.openmessaging.storage.dledger.client.DLedgerClient; 23 | import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; 24 | import io.openmessaging.storage.dledger.protocol.LeadershipTransferResponse; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | @Parameters(commandDescription = "Leadership transfer") 29 | public class LeadershipTransferCommand extends BaseCommand { 30 | 31 | private static final Logger LOGGER = LoggerFactory.getLogger(LeadershipTransferCommand.class); 32 | 33 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 34 | private String group = "default"; 35 | 36 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 37 | private String peers = "n0-localhost:20911"; 38 | 39 | @Parameter(names = {"--leader", "-l"}, description = "set the current leader manually") 40 | private String leaderId; 41 | 42 | @Parameter(names = {"--transfereeId", "-t"}, description = "Node try to be the new leader") 43 | private String transfereeId = "n0"; 44 | 45 | @Parameter(names = {"--term"}, description = "current term") 46 | private long term; 47 | 48 | @Override 49 | public void doCommand() { 50 | DLedgerClient dLedgerClient = new DLedgerClient(group, peers); 51 | dLedgerClient.startup(); 52 | LeadershipTransferResponse response = dLedgerClient.leadershipTransfer(leaderId, transfereeId, term); 53 | LOGGER.info("LeadershipTransfer code={}, Result:{}", DLedgerResponseCode.valueOf(response.getCode()), 54 | JSON.toJSONString(response)); 55 | dLedgerClient.shutdown(); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterDLedger.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerConfig; 20 | import io.openmessaging.storage.dledger.DLedgerServer; 21 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadProcessor; 22 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteProcessor; 23 | import io.openmessaging.storage.dledger.metrics.DLedgerMetricsManager; 24 | import java.util.Arrays; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | public class RegisterDLedger { 29 | 30 | private static final Logger LOGGER = LoggerFactory.getLogger(RegisterDLedger.class); 31 | 32 | public static void bootstrap(DLedgerConfig dLedgerConfig) { 33 | DLedgerServer dLedgerServer = new DLedgerServer(dLedgerConfig); 34 | dLedgerServer.registerStateMachine(new RegisterStateMachine()); 35 | dLedgerServer.registerUserDefineProcessors( 36 | Arrays.asList(new RegisterWriteProcessor(dLedgerServer), new RegisterReadProcessor(dLedgerServer)) 37 | ); 38 | DLedgerMetricsManager.defaultInit(dLedgerServer); 39 | dLedgerServer.startup(); 40 | LOGGER.info("RegisterDLedger started"); 41 | Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { 42 | private volatile boolean hasShutdown = false; 43 | 44 | @Override 45 | public void run() { 46 | synchronized (this) { 47 | LOGGER.info("Shutdown hook was invoked"); 48 | if (!this.hasShutdown) { 49 | this.hasShutdown = true; 50 | long beginTime = System.currentTimeMillis(); 51 | dLedgerServer.shutdown(); 52 | long consumingTimeTotal = System.currentTimeMillis() - beginTime; 53 | LOGGER.info("Shutdown hook over, consuming total time(ms): {}", consumingTimeTotal); 54 | } 55 | } 56 | } 57 | }, "ShutdownHook")); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/RegisterSnapshotFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register; 18 | 19 | import io.openmessaging.storage.dledger.utils.IOUtils; 20 | import java.io.IOException; 21 | import java.util.Map; 22 | import org.slf4j.Logger; 23 | import org.slf4j.LoggerFactory; 24 | 25 | public class RegisterSnapshotFile { 26 | private static Logger logger = LoggerFactory.getLogger(RegisterSnapshotFile.class); 27 | 28 | private final String snapshotStorePath; 29 | 30 | public RegisterSnapshotFile(String snapshotStorePath) { 31 | this.snapshotStorePath = snapshotStorePath; 32 | } 33 | 34 | public boolean save(final Map register) { 35 | try { 36 | IOUtils.object2File(snapshotStorePath, register); 37 | } catch (IOException e) { 38 | logger.error("Unable to save snapshot data", e); 39 | return false; 40 | } 41 | return true; 42 | } 43 | 44 | public Map load() throws IOException, ClassNotFoundException { 45 | Object object = IOUtils.file2Object(this.snapshotStorePath); 46 | if (object instanceof Map) { 47 | Map map = (Map) object; 48 | return map; 49 | } else { 50 | throw new IOException("Unable to load data from file to map"); 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/client/RegisterDLedgerClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.client; 18 | 19 | import io.openmessaging.storage.dledger.common.ReadMode; 20 | import io.openmessaging.storage.dledger.client.DLedgerClient; 21 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadRequest; 22 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadResponse; 23 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteRequest; 24 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteResponse; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | public class RegisterDLedgerClient { 29 | 30 | private static final Logger LOGGER = LoggerFactory.getLogger(RegisterDLedgerClient.class); 31 | 32 | private final DLedgerClient client; 33 | 34 | public RegisterDLedgerClient(String group, String peers) { 35 | this.client = new DLedgerClient(group, peers); 36 | } 37 | 38 | public void startup() { 39 | client.startup(); 40 | } 41 | 42 | public void shutdown() { 43 | client.shutdown(); 44 | } 45 | 46 | public RegisterWriteResponse write(int key, int value) { 47 | RegisterWriteRequest request = new RegisterWriteRequest(key, value); 48 | return client.invokeUserDefineRequest(request, RegisterWriteResponse.class, true); 49 | } 50 | 51 | public RegisterReadResponse read(int key) { 52 | return this.read(key, ReadMode.RAFT_LOG_READ); 53 | } 54 | 55 | public RegisterReadResponse read(int key, String readModeStr) { 56 | ReadMode readMode = ReadMode.RAFT_LOG_READ; 57 | try { 58 | readMode = ReadMode.valueOf(readModeStr); 59 | } catch (Exception ignore) { 60 | LOGGER.error("Error readMode string: {}, use default readMode: ", readModeStr, ReadMode.RAFT_LOG_READ); 61 | } 62 | return this.read(key, readMode); 63 | } 64 | 65 | public RegisterReadResponse read(int key, ReadMode readMode) { 66 | RegisterReadRequest request = new RegisterReadRequest(key, readMode); 67 | return client.invokeUserDefineRequest(request, RegisterReadResponse.class, true); 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/command/BenchmarkCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.command; 18 | 19 | import com.beust.jcommander.Parameter; 20 | import com.beust.jcommander.Parameters; 21 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 22 | import io.openmessaging.storage.dledger.example.register.RegisterBenchmark; 23 | import org.slf4j.Logger; 24 | import org.slf4j.LoggerFactory; 25 | 26 | @Parameters(commandDescription = "Benchmark the RegisterDLedger") 27 | public class BenchmarkCommand extends BaseCommand { 28 | 29 | private static Logger logger = LoggerFactory.getLogger(BenchmarkCommand.class); 30 | 31 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 32 | private String group = "default"; 33 | 34 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 35 | private String peers = "n0-localhost:20911"; 36 | 37 | @Parameter(names = {"--clientNum", "-c"}, description = "Client number") 38 | private int clientNum = 10; 39 | 40 | @Parameter(names = {"--ops", "-o"}, description = "Operation num per client") 41 | private long opsPerClient = 10000; 42 | 43 | @Parameter(names = {"--benchmarkType", "-t"}, description = "Benchmark type]") 44 | private RegisterBenchmark.BenchmarkType benchmarkType = RegisterBenchmark.BenchmarkType.Write; 45 | 46 | @Override 47 | public void doCommand() { 48 | RegisterBenchmark benchmark = new RegisterBenchmark(group, peers, clientNum, opsPerClient, benchmarkType); 49 | try { 50 | benchmark.start(); 51 | } catch (Exception e) { 52 | logger.error("benchmark error", e); 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/command/ReadCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.command; 18 | 19 | import com.alibaba.fastjson.JSON; 20 | import com.beust.jcommander.Parameter; 21 | import com.beust.jcommander.Parameters; 22 | import io.openmessaging.storage.dledger.common.ReadMode; 23 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 24 | import io.openmessaging.storage.dledger.example.register.client.RegisterDLedgerClient; 25 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterReadResponse; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | @Parameters(commandDescription = "Read a key from the RegisterDLedger") 30 | public class ReadCommand extends BaseCommand { 31 | 32 | private static Logger logger = LoggerFactory.getLogger(ReadCommand.class); 33 | 34 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 35 | private String group = "default"; 36 | 37 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 38 | private String peers = "n0-localhost:20911"; 39 | 40 | @Parameter(names = {"--key", "-k"}, description = "The key to read") 41 | private int key = 13; 42 | 43 | @Parameter(names = {"--read-mode", "-r"}, description = "Read mode") 44 | private ReadMode readMode = ReadMode.RAFT_LOG_READ; 45 | 46 | 47 | @Override 48 | public void doCommand() { 49 | RegisterDLedgerClient client = new RegisterDLedgerClient(group, peers); 50 | client.startup(); 51 | RegisterReadResponse response = client.read(key, readMode); 52 | logger.info("Get Result:{}", JSON.toJSONString(response)); 53 | client.shutdown(); 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/command/WriteCommand.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.command; 18 | 19 | import com.beust.jcommander.Parameter; 20 | import com.beust.jcommander.Parameters; 21 | import io.openmessaging.storage.dledger.example.common.command.BaseCommand; 22 | import io.openmessaging.storage.dledger.example.register.client.RegisterDLedgerClient; 23 | import io.openmessaging.storage.dledger.example.register.protocol.RegisterWriteResponse; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | @Parameters(commandDescription = "Write a key-value pair to RegisterDLedger") 28 | public class WriteCommand extends BaseCommand { 29 | private static Logger logger = LoggerFactory.getLogger(WriteCommand.class); 30 | 31 | @Parameter(names = {"--group", "-g"}, description = "Group of this server") 32 | private String group = "default"; 33 | 34 | @Parameter(names = {"--peers", "-p"}, description = "Peer info of this server") 35 | private String peers = "n0-localhost:20911"; 36 | 37 | @Parameter(names = {"--key", "-k"}, description = "the key to set") 38 | private int key = 13; 39 | 40 | @Parameter(names = {"--value", "-v"}, description = "the value to set") 41 | private int value = 31; 42 | 43 | 44 | @Override 45 | public void doCommand() { 46 | RegisterDLedgerClient client = new RegisterDLedgerClient(group, peers); 47 | client.startup(); 48 | RegisterWriteResponse response = client.write(key, value); 49 | logger.info("Write Result Code:{}", response.getCode()); 50 | client.shutdown(); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerServer; 20 | import io.openmessaging.storage.dledger.common.ReadClosure; 21 | import io.openmessaging.storage.dledger.common.Status; 22 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; 23 | import io.openmessaging.storage.dledger.example.register.RegisterStateMachine; 24 | import java.util.concurrent.CompletableFuture; 25 | 26 | public class RegisterReadProcessor extends UserDefineProcessor { 27 | 28 | public RegisterReadProcessor(DLedgerServer server) { 29 | super(server); 30 | } 31 | @Override 32 | public CompletableFuture handleRequest(RegisterReadRequest registerReadRequest) { 33 | Integer key = registerReadRequest.getKey(); 34 | RegisterReadResponse response = new RegisterReadResponse(); 35 | response.setKey(key); 36 | CompletableFuture future = new CompletableFuture<>(); 37 | ReadClosure closure = new ReadClosure() { 38 | @Override 39 | public void done(Status status) { 40 | if (status.isOk()) { 41 | RegisterStateMachine registerStateMachine = (RegisterStateMachine) dLedgerServer.getStateMachine(); 42 | Integer value = registerStateMachine.getValue(key); 43 | response.setValue(value); 44 | future.complete(response); 45 | } else { 46 | response.setCode(status.code.getCode()); 47 | future.complete(response); 48 | } 49 | } 50 | }; 51 | dLedgerServer.handleRead(registerReadRequest.getReadMode(), closure); 52 | return future; 53 | } 54 | 55 | @Override 56 | public Integer getRequestTypeCode() { 57 | return RegisterRequestTypeCode.READ.ordinal(); 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.common.ReadMode; 20 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; 21 | 22 | public class RegisterReadRequest extends UserDefineRequest { 23 | 24 | private final Integer key; 25 | 26 | private ReadMode readMode = ReadMode.RAFT_LOG_READ; 27 | 28 | public RegisterReadRequest(int key) { 29 | this.key = key; 30 | } 31 | 32 | public RegisterReadRequest(int key, ReadMode readMode) { 33 | this.key = key; 34 | this.readMode = readMode; 35 | } 36 | 37 | public void setReadMode(ReadMode readMode) { 38 | this.readMode = readMode; 39 | } 40 | 41 | public ReadMode getReadMode() { 42 | return readMode; 43 | } 44 | 45 | public Integer getKey() { 46 | return key; 47 | } 48 | 49 | @Override 50 | public int getRequestTypeCode() { 51 | return RegisterRequestTypeCode.READ.ordinal(); 52 | } 53 | 54 | @Override 55 | public String toString() { 56 | return "RegisterReadRequest{" + 57 | "key=" + key + 58 | ", readMode=" + readMode + 59 | ", group='" + group + '\'' + 60 | ", remoteId='" + remoteId + '\'' + 61 | ", localId='" + localId + '\'' + 62 | ", code=" + code + 63 | ", leaderId='" + leaderId + '\'' + 64 | ", term=" + term + 65 | '}'; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterReadResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineResponse; 20 | 21 | public class RegisterReadResponse extends UserDefineResponse { 22 | 23 | private Integer key; 24 | 25 | private Integer value; 26 | 27 | public void setKey(Integer key) { 28 | this.key = key; 29 | } 30 | 31 | public void setValue(Integer value) { 32 | this.value = value; 33 | } 34 | 35 | public Integer getKey() { 36 | return key; 37 | } 38 | 39 | public Integer getValue() { 40 | return value; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterRequestTypeCode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | public enum RegisterRequestTypeCode { 20 | WRITE, 21 | READ 22 | } 23 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteProcessor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerServer; 20 | import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode; 21 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineProcessor; 22 | import io.openmessaging.storage.dledger.utils.BytesUtil; 23 | import java.util.concurrent.CompletableFuture; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | public class RegisterWriteProcessor extends UserDefineProcessor { 28 | 29 | private static final Logger LOGGER = LoggerFactory.getLogger(UserDefineProcessor.class); 30 | 31 | public RegisterWriteProcessor(DLedgerServer dLedgerServer) { 32 | super(dLedgerServer); 33 | } 34 | 35 | @Override 36 | public CompletableFuture handleRequest(RegisterWriteRequest request) { 37 | Integer key = request.getKey(); 38 | Integer value = request.getValue(); 39 | byte[] body = encodeWriteEvent(key, value); 40 | return dLedgerServer.appendAsLeader(body).thenApply(appendResp -> { 41 | RegisterWriteResponse response = new RegisterWriteResponse(); 42 | response.setTerm(appendResp.getTerm()); 43 | response.setCode(appendResp.getCode()); 44 | response.setLeaderId(appendResp.getLeaderId()); 45 | return response; 46 | }).exceptionally(e -> { 47 | LOGGER.error("Failed to append bytes to dLedger", e); 48 | RegisterWriteResponse response = new RegisterWriteResponse(); 49 | response.setCode(DLedgerResponseCode.UNKNOWN.getCode()); 50 | return response; 51 | }); 52 | } 53 | 54 | @Override 55 | public Integer getRequestTypeCode() { 56 | return RegisterRequestTypeCode.WRITE.ordinal(); 57 | } 58 | 59 | private byte[] encodeWriteEvent(Integer key, Integer value) { 60 | byte[] keyBytes = BytesUtil.intToBytes(key); 61 | byte[] valueBytes = BytesUtil.intToBytes(value); 62 | byte[] afterCodingBytes = new byte[8]; 63 | System.arraycopy(keyBytes, 0, afterCodingBytes, 0, 4); 64 | System.arraycopy(valueBytes, 0, afterCodingBytes, 4, 4); 65 | return afterCodingBytes; 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineRequest; 20 | 21 | public class RegisterWriteRequest extends UserDefineRequest { 22 | 23 | private Integer key; 24 | 25 | private Integer value; 26 | 27 | public RegisterWriteRequest(int key, int value) { 28 | this.key = key; 29 | this.value = value; 30 | } 31 | 32 | public void setKey(Integer key) { 33 | this.key = key; 34 | } 35 | 36 | public void setValue(Integer value) { 37 | this.value = value; 38 | } 39 | 40 | public Integer getKey() { 41 | return key; 42 | } 43 | 44 | public Integer getValue() { 45 | return value; 46 | } 47 | 48 | @Override 49 | public int getRequestTypeCode() { 50 | return RegisterRequestTypeCode.WRITE.ordinal(); 51 | } 52 | 53 | @Override 54 | public String toString() { 55 | return "RegisterWriteRequest{" + 56 | "key=" + key + 57 | ", value=" + value + 58 | ", group='" + group + '\'' + 59 | ", remoteId='" + remoteId + '\'' + 60 | ", localId='" + localId + '\'' + 61 | ", code=" + code + 62 | ", leaderId='" + leaderId + '\'' + 63 | ", term=" + term + 64 | '}'; 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /example/src/main/java/io/openmessaging/storage/dledger/example/register/protocol/RegisterWriteResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.register.protocol; 18 | 19 | import io.openmessaging.storage.dledger.protocol.userdefine.UserDefineResponse; 20 | 21 | public class RegisterWriteResponse extends UserDefineResponse { 22 | } 23 | -------------------------------------------------------------------------------- /example/src/main/resources/appender/config.example.error.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10001;n2-127.0.0.1:10002 5 | preferredLeaderIds: n0 6 | - group: g1 7 | selfId: a0 8 | peers: a0-127.0.0.1:20000;a1-127.0.0.1:20001;a2-127.0.0.1:20002 9 | preferredLeaderIds: a1 10 | - group: g2 11 | selfId: b0 12 | peers: b0-127.0.0.1:30000;b1-127.0.0.1:30001;b2-127.0.0.1:30002 13 | preferredLeaderIds: b2 14 | -------------------------------------------------------------------------------- /example/src/main/resources/appender/config.example.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10001;n2-127.0.0.1:10002 5 | preferredLeaderIds: n0 6 | - group: g1 7 | selfId: a0 8 | peers: a0-127.0.0.1:10000;a1-127.0.0.1:10001;a2-127.0.0.1:10002 9 | preferredLeaderIds: a1 10 | - group: g2 11 | selfId: b0 12 | peers: b0-127.0.0.1:10000;b1-127.0.0.1:10001;b2-127.0.0.1:10002 13 | preferredLeaderIds: b2 14 | -------------------------------------------------------------------------------- /example/src/test/java/io/openmessaging/storage/dledger/example/appender/CommandTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.example.appender; 18 | 19 | import com.beust.jcommander.JCommander; 20 | import io.openmessaging.storage.dledger.example.appender.command.ConfigCommand; 21 | import io.openmessaging.storage.dledger.proxy.DLedgerProxyConfig; 22 | import io.openmessaging.storage.dledger.proxy.util.ConfigUtils; 23 | import org.junit.jupiter.api.Assertions; 24 | import org.junit.jupiter.api.Test; 25 | 26 | 27 | public class CommandTest { 28 | 29 | @Test 30 | public void TestParseArguments() { 31 | ConfigCommand configCommand = new ConfigCommand(); 32 | JCommander.Builder builder = JCommander.newBuilder().addObject(configCommand); 33 | JCommander jc = builder.build(); 34 | jc.parse("-c", "./src/test/resources/config.example.yaml"); 35 | Assertions.assertEquals("./src/test/resources/config.example.yaml", configCommand.getConfigPath()); 36 | } 37 | 38 | @Test 39 | public void TestGenerateProxyConfig() { 40 | ConfigCommand configCommand = new ConfigCommand(); 41 | JCommander.Builder builder = JCommander.newBuilder().addObject(configCommand); 42 | JCommander jc = builder.build(); 43 | jc.parse("-c", "./src/test/resources/config.example.yaml"); 44 | Assertions.assertEquals("./src/test/resources/config.example.yaml", configCommand.getConfigPath()); 45 | try { 46 | DLedgerProxyConfig config = ConfigUtils.parseDLedgerProxyConfig(configCommand.getConfigPath()); 47 | Assertions.assertNotNull(config); 48 | Assertions.assertEquals(3, config.getConfigs().size()); 49 | String s = ""; 50 | for (int i = 0; i < config.getConfigs().size(); i++) { 51 | Assertions.assertEquals("127.0.0.1:10000", config.getConfigs().get(i).getSelfAddress()); 52 | Assertions.assertEquals("g" + i, config.getConfigs().get(i).getGroup()); 53 | s += config.getConfigs().get(i).getSelfId(); 54 | } 55 | Assertions.assertEquals("n0a0b0", s); 56 | } catch (Exception e) { 57 | Assertions.assertNotNull(e); 58 | } 59 | } 60 | 61 | } 62 | -------------------------------------------------------------------------------- /jepsen/build.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | # build dledger 19 | cd ../; 20 | mvn clean install -DskipTests; 21 | cp example/target/dledger-example.jar jepsen/node-deploy/; 22 | rm jepsen/dledger-jepsen.tar.gz; 23 | chmod +x jepsen/node-deploy/startup.sh; 24 | chmod +x jepsen/node-deploy/stop.sh; 25 | chmod +x jepsen/node-deploy/stop_dropcaches.sh; 26 | tar zcvf jepsen/dledger-jepsen-node.tar.gz jepsen/node-deploy/dledger-example.jar jepsen/node-deploy/startup.sh jepsen/node-deploy/stop.sh jepsen/node-deploy/stop_dropcaches.sh; 27 | 28 | # build jepsen test 29 | cd jepsen; 30 | lein uberjar; 31 | chmod +x jepsen.sh; 32 | cd ../; 33 | tar zcvf jepsen/dledger-jepsen-control.tar.gz jepsen/jepsen.sh jepsen/nodes jepsen/target/openmessaging-dledger-jepsen-exec.jar; -------------------------------------------------------------------------------- /jepsen/build_deploy.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | sh ./build.sh ; 19 | control run dledger-control control-deploy; 20 | control run dledger-node node-deploy; -------------------------------------------------------------------------------- /jepsen/control.clj: -------------------------------------------------------------------------------- 1 | ; Licensed to the Apache Software Foundation (ASF) under one or more 2 | ; contributor license agreements. See the NOTICE file distributed with 3 | ; this work for additional information regarding copyright ownership. 4 | ; The ASF licenses this file to You under the Apache License, Version 2.0 5 | ; (the "License"); you may not use this file except in compliance with 6 | ; the License. You may obtain a copy of the License at 7 | ; 8 | ; http://www.apache.org/licenses/LICENSE-2.0 9 | ; 10 | ; Unless required by applicable law or agreed to in writing, software 11 | ; distributed under the License is distributed on an "AS IS" BASIS, 12 | ; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | ; See the License for the specific language governing permissions and 14 | ; limitations under the License. 15 | 16 | 17 | (defcluster :dledger-node 18 | :clients [{:host "n0" :user "root"} 19 | {:host "n1" :user "root"} 20 | {:host "n2" :user "root"} 21 | {:host "n3" :user "root"} 22 | {:host "n4" :user "root"}]) 23 | 24 | (defcluster :dledger-control 25 | :clients [{:host "n5" :user "root"}]) 26 | 27 | (deftask :date "echo date on cluster" [] 28 | (ssh "date")) 29 | 30 | (deftask :node-deploy [] 31 | (ssh 32 | (run 33 | (cd "~" 34 | (run "rm -rf jepsen/") 35 | (run "rm -rf dledger-jepsen-node.tar.gz")))) 36 | (scp "dledger-jepsen-node.tar.gz" "~/") 37 | (ssh 38 | (run 39 | (cd "~" 40 | (run "tar zxvf dledger-jepsen-node.tar.gz") 41 | (run "rm -rf dledger-jepsen-node.tar.gz"))))) 42 | 43 | (deftask :control-deploy [] 44 | (ssh 45 | (run 46 | (cd "~" 47 | (run "rm -rf jepsen/") 48 | (run "rm -rf dledger-jepsen-control.tar.gz")))) 49 | (scp "dledger-jepsen-control.tar.gz" "~/") 50 | (ssh 51 | (run 52 | (cd "~" 53 | (run "tar zxvf dledger-jepsen-control.tar.gz") 54 | (run "rm -rf dledger-jepsen-control.tar.gz"))))) -------------------------------------------------------------------------------- /jepsen/jepsen.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | java -jar target/openmessaging-dledger-jepsen-exec.jar test $@ -------------------------------------------------------------------------------- /jepsen/node-deploy/startup.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | nohup java -jar ./dledger-example.jar register $@ >> register-dledger.log 2>&1 & -------------------------------------------------------------------------------- /jepsen/node-deploy/stop.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | pid=`ps -ef |grep dledger-example |grep java |awk -F' ' '{print $2}'` 19 | if [ "$pid" != "" ] 20 | then 21 | echo "kill $pid" 22 | kill $pid 23 | fi -------------------------------------------------------------------------------- /jepsen/node-deploy/stop_dropcaches.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one or more 4 | # contributor license agreements. See the NOTICE file distributed with 5 | # this work for additional information regarding copyright ownership. 6 | # The ASF licenses this file to You under the Apache License, Version 2.0 7 | # (the "License"); you may not use this file except in compliance with 8 | # the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | 18 | pid=`ps -ef |grep dledger-example |grep java |awk -F' ' '{print $2}'` 19 | if [ "$pid" != "" ] 20 | then 21 | echo "kill $pid" 22 | kill $pid 23 | fi 24 | 25 | # To free pagecache, dentries and inodes 26 | echo 3 >/proc/sys/vm/drop_caches -------------------------------------------------------------------------------- /jepsen/nodes: -------------------------------------------------------------------------------- 1 | n0 2 | n1 3 | n2 4 | n3 5 | n4 -------------------------------------------------------------------------------- /jepsen/project.clj: -------------------------------------------------------------------------------- 1 | ; Licensed to the Apache Software Foundation (ASF) under one or more 2 | ; contributor license agreements. See the NOTICE file distributed with 3 | ; this work for additional information regarding copyright ownership. 4 | ; The ASF licenses this file to You under the Apache License, Version 2.0 5 | ; (the "License"); you may not use this file except in compliance with 6 | ; the License. You may obtain a copy of the License at 7 | ; 8 | ; http://www.apache.org/licenses/LICENSE-2.0 9 | ; 10 | ; Unless required by applicable law or agreed to in writing, software 11 | ; distributed under the License is distributed on an "AS IS" BASIS, 12 | ; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | ; See the License for the specific language governing permissions and 14 | ; limitations under the License. 15 | 16 | 17 | (defproject openmessaging-dledger-jepsen "0.0.1-SNAPSHOT" 18 | :descritopn "A jepsen test for DLedger" 19 | :license {:name "Apache License 2.0"} 20 | :main io.openmessaging.storage.dledger.jepsen.core 21 | :dependencies [[org.clojure/clojure "1.10.0"] 22 | [jepsen "0.1.15-SNAPSHOT"] 23 | [io.openmessaging.storage/dledger-example "0.3.3-SNAPSHOT"]] 24 | :aot [io.openmessaging.storage.dledger.jepsen.core] 25 | :source-paths ["src" "src/main/clojure"] 26 | :jar-name "openmessaging-dledger-jepsen.jar" 27 | :uberjar-name "openmessaging-dledger-jepsen-exec.jar") -------------------------------------------------------------------------------- /proxy/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | dledger-all 4 | io.openmessaging.storage 5 | 0.3.3-SNAPSHOT 6 | 7 | 4.0.0 8 | 9 | dledger-proxy 10 | 11 | 12 | UTF-8 13 | 14 | 15 | 16 | 17 | io.openmessaging.storage 18 | dledger 19 | 20 | 21 | org.yaml 22 | snakeyaml 23 | 24 | 25 | org.slf4j 26 | slf4j-simple 27 | test 28 | 29 | 30 | org.mockito 31 | mockito-core 32 | test 33 | 34 | 35 | org.junit.jupiter 36 | junit-jupiter-api 37 | test 38 | 39 | 40 | 41 | -------------------------------------------------------------------------------- /proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxyConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.proxy; 18 | 19 | 20 | import io.openmessaging.storage.dledger.DLedgerConfig; 21 | 22 | import java.util.List; 23 | 24 | public class DLedgerProxyConfig { 25 | 26 | private List configs; 27 | 28 | public List getConfigs() { 29 | return configs; 30 | } 31 | 32 | public void setConfigs(List configs) { 33 | this.configs = configs; 34 | } 35 | 36 | @Override 37 | public String toString() { 38 | return "DLedgerProxyConfig{" + 39 | "configs=" + configs + 40 | '}'; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /proxy/src/main/java/io/openmessaging/storage/dledger/proxy/DLedgerProxyConfigListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package io.openmessaging.storage.dledger.proxy; 17 | 18 | import io.openmessaging.storage.dledger.DLedgerConfig; 19 | 20 | public interface DLedgerProxyConfigListener { 21 | 22 | void onDLedgerConfigChange(DLedgerConfig addOrRemovedConfig, ConfigChangeEvent event); 23 | enum ConfigChangeEvent { 24 | ADD, 25 | REMOVED 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /proxy/src/main/java/io/openmessaging/storage/dledger/proxy/util/ConfigUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.proxy.util; 18 | 19 | import io.openmessaging.storage.dledger.DLedgerConfig; 20 | import io.openmessaging.storage.dledger.proxy.DLedgerProxyConfig; 21 | import java.nio.file.Files; 22 | import java.nio.file.Paths; 23 | import org.yaml.snakeyaml.Yaml; 24 | import org.yaml.snakeyaml.constructor.Constructor; 25 | 26 | import java.io.InputStream; 27 | 28 | public class ConfigUtils { 29 | 30 | public static DLedgerProxyConfig parseDLedgerProxyConfig(final String path) throws Exception { 31 | DLedgerProxyConfig config = null; 32 | final Yaml yaml = new Yaml(new Constructor(DLedgerProxyConfig.class)); 33 | InputStream inputStream = Files.newInputStream(Paths.get(path)); 34 | config = yaml.load(inputStream); 35 | inputStream.close(); 36 | if (config == null) { 37 | throw new IllegalArgumentException("DLedger Config doesn't exist"); 38 | } 39 | if (!checkProxyConfig(config)) { 40 | throw new IllegalArgumentException("DLedger servers in DLedger Config don't have the same port"); 41 | } 42 | return config; 43 | } 44 | 45 | public static boolean checkProxyConfig(DLedgerProxyConfig dLedgerProxyConfig) { 46 | // check same port 47 | String bindPort = null; 48 | for (DLedgerConfig config : dLedgerProxyConfig.getConfigs()) { 49 | config.init(); 50 | String[] split = config.getSelfAddress().split(":"); 51 | if (bindPort == null) { 52 | bindPort = split[1]; 53 | } else { 54 | if (!bindPort.equals(split[1])) { 55 | return false; 56 | } 57 | } 58 | } 59 | return true; 60 | } 61 | 62 | } 63 | -------------------------------------------------------------------------------- /proxy/src/test/java/io/openmessaging/storage/dledger/proxy/util/ConfigUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.proxy.util; 18 | 19 | import io.openmessaging.storage.dledger.proxy.DLedgerProxyConfig; 20 | import io.openmessaging.storage.dledger.proxy.util.ConfigUtils; 21 | import java.nio.file.NoSuchFileException; 22 | import org.junit.jupiter.api.Assertions; 23 | import org.junit.jupiter.api.Test; 24 | 25 | public class ConfigUtilsTest { 26 | 27 | @Test 28 | public void TestConfigUtilsInit() throws Exception { 29 | DLedgerProxyConfig config = ConfigUtils.parseDLedgerProxyConfig("./src/test/resources/config.example.yaml"); 30 | Assertions.assertEquals(3, config.getConfigs().size()); 31 | for (int i = 0; i < config.getConfigs().size(); i++) { 32 | Assertions.assertEquals("127.0.0.1:10000", config.getConfigs().get(i).getSelfAddress()); 33 | Assertions.assertEquals("g" + i, config.getConfigs().get(i).getGroup()); 34 | } 35 | } 36 | 37 | @Test 38 | public void TestConfigUtilsInitErrorConfig() { 39 | DLedgerProxyConfig config = null; 40 | try { 41 | config = ConfigUtils.parseDLedgerProxyConfig("./src/test/resources/config.example.error.yaml"); 42 | } catch (Exception e) { 43 | Assertions.assertNotNull(e); 44 | Assertions.assertEquals("DLedger servers in DLedger Config don't have the same port", e.getMessage()); 45 | } 46 | Assertions.assertNull(config); 47 | } 48 | 49 | @Test 50 | public void TestConfigFileNotExist() { 51 | DLedgerProxyConfig config = null; 52 | try { 53 | config = ConfigUtils.parseDLedgerProxyConfig("./lcy.txt"); 54 | }catch (Exception e) { 55 | Assertions.assertNotNull(e); 56 | Assertions.assertEquals(NoSuchFileException.class, e.getClass()); 57 | } 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /proxy/src/test/java/io/openmessaging/storage/dledger/proxy/util/FileTestUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017-2022 The DLedger Authors. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * https://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package io.openmessaging.storage.dledger.proxy.util; 18 | 19 | import io.openmessaging.storage.dledger.utils.IOUtils; 20 | import java.io.File; 21 | import java.io.IOException; 22 | import java.util.UUID; 23 | 24 | public class FileTestUtil { 25 | 26 | public static final String TEST_BASE = File.separator + "tmp" + File.separator + "dledgerteststore"; 27 | 28 | public static String createTestDir() { 29 | return createTestDir(null); 30 | } 31 | 32 | public static String createTestDir(String prefix) { 33 | if (prefix == null) { 34 | prefix = "test"; 35 | } 36 | String baseDir = TEST_BASE + File.separator + prefix + "-" + UUID.randomUUID().toString(); 37 | final File file = new File(baseDir); 38 | if (file.exists()) { 39 | System.exit(1); 40 | } 41 | return baseDir; 42 | } 43 | 44 | public static void deleteFile(String fileName) throws IOException { 45 | IOUtils.deleteFile(new File(fileName)); 46 | } 47 | 48 | public static void deleteFile(File file) throws IOException { 49 | IOUtils.deleteFile(file); 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /proxy/src/test/resources/config.example.error.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10001;n2-127.0.0.1:10002 5 | preferredLeaderIds: n0 6 | - group: g1 7 | selfId: a0 8 | peers: a0-127.0.0.1:20000;a1-127.0.0.1:20001;a2-127.0.0.1:20002 9 | preferredLeaderIds: a1 10 | - group: g2 11 | selfId: b0 12 | peers: b0-127.0.0.1:30000;b1-127.0.0.1:30001;b2-127.0.0.1:30002 13 | preferredLeaderIds: b2 14 | -------------------------------------------------------------------------------- /proxy/src/test/resources/config.example.yaml: -------------------------------------------------------------------------------- 1 | configs: 2 | - group: g0 3 | selfId: n0 4 | peers: n0-127.0.0.1:10000;n1-127.0.0.1:10001;n2-127.0.0.1:10002 5 | preferredLeaderIds: n0 6 | - group: g1 7 | selfId: a0 8 | peers: a0-127.0.0.1:10000;a1-127.0.0.1:10001;a2-127.0.0.1:10002 9 | preferredLeaderIds: a1 10 | - group: g2 11 | selfId: b0 12 | peers: b0-127.0.0.1:10000;b1-127.0.0.1:10001;b2-127.0.0.1:10002 13 | preferredLeaderIds: b2 14 | -------------------------------------------------------------------------------- /script/deploy-three-register.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Copyright 2017-2022 The DLedger Authors. 4 | # 5 | # Licensed under the Apache License, Version 2.0 (the "License"); 6 | # you may not use this file except in compliance with the License. 7 | # You may obtain a copy of the License at 8 | # 9 | # https://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | nohup java -jar ../example/target/dledger-example.jar register --peers "n0-localhost:20911;n1-localhost:20912;n2-localhost:20913" --id n0 --metrics-exporter-type PROM --metrics-prom-export-port 5557 > register-n0.log 2>&1 & 19 | 20 | nohup java -jar ../example/target/dledger-example.jar register --peers "n0-localhost:20911;n1-localhost:20912;n2-localhost:20913" --id n1 --metrics-exporter-type PROM --metrics-prom-export-port 5558 > register-n1.log 2>&1 & 21 | 22 | nohup java -jar ../example/target/dledger-example.jar register --peers "n0-localhost:20911;n1-localhost:20912;n2-localhost:20913" --id n2 --metrics-exporter-type PROM --metrics-prom-export-port 5559 > register-n2.log 2>&1 & 23 | 24 | -------------------------------------------------------------------------------- /style/copyright/Apache.xml: -------------------------------------------------------------------------------- 1 | 16 | 17 | 18 | 19 | 23 | -------------------------------------------------------------------------------- /style/copyright/profiles_settings.xml: -------------------------------------------------------------------------------- 1 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 26 | 27 | 30 | 31 | 34 | 35 | 38 | 39 | 42 | 43 | 46 | 47 | 50 | 51 | 54 | 55 | 58 | 59 | 62 | 63 | --------------------------------------------------------------------------------