├── .asf.yaml ├── .github ├── pull_request_template.md └── workflows │ ├── check.yaml │ ├── ci.yaml │ ├── post-commit.yaml │ └── repeat-test.yaml ├── .gitignore ├── .idea └── vcs.xml ├── .mvn ├── develocity.xml ├── extensions.xml └── wrapper │ └── maven-wrapper.properties ├── BUILDING.md ├── DEPLOY.md ├── LICENSE ├── NOTICE ├── README.md ├── dev-support ├── checks │ ├── _mvn_unit_report.sh │ ├── _summary.sh │ ├── author.sh │ ├── build.sh │ ├── checkstyle.sh │ ├── compile.sh │ ├── coverage.sh │ ├── findbugs.sh │ ├── rat.sh │ ├── release.sh │ ├── repro.sh │ ├── sonar.sh │ └── unit.sh ├── checkstyle.xml ├── docker │ └── Dockerfile ├── find_maven.sh ├── intellij │ ├── install-runconfig.sh │ └── runConfigurations │ │ ├── ExampleServer1.xml │ │ ├── ExampleServer2.xml │ │ └── ExampleServer3.xml ├── make_rc.sh ├── run-test-repeatedly.sh └── vagrant │ ├── .gitignore │ ├── README.md │ ├── Vagrantfile │ ├── bin │ ├── start_ratis_load_gen.sh │ └── start_ratis_server.sh │ ├── docs │ ├── vagrantfile.bpmn │ └── vagrantfile_bpmn.svg │ ├── namazu_configs │ └── hdd_config.toml │ ├── run_all_tests.sh │ └── screenrcs │ ├── namazu_hdd_screenrc │ ├── ratis_ratis-hdd-slowdown_screenrc │ └── ratis_ratis-server_screenrc ├── mvnw ├── mvnw.cmd ├── pom.xml ├── ratis-assembly ├── pom.xml └── src │ └── main │ ├── assembly │ ├── bin.xml │ └── src.xml │ └── resources │ ├── LICENSE │ ├── NOTICE │ └── README.md ├── ratis-client ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── ratis │ └── client │ ├── AsyncRpcApi.java │ ├── ClientFactory.java │ ├── DataStreamClient.java │ ├── DataStreamClientFactory.java │ ├── DataStreamClientRpc.java │ ├── DataStreamOutputRpc.java │ ├── DataStreamRpcApi.java │ ├── DisabledDataStreamClientFactory.java │ ├── RaftClient.java │ ├── RaftClientConfigKeys.java │ ├── RaftClientRpc.java │ ├── api │ ├── AdminApi.java │ ├── AsyncApi.java │ ├── BlockingApi.java │ ├── DataStreamApi.java │ ├── DataStreamOutput.java │ ├── GroupManagementApi.java │ ├── LeaderElectionManagementApi.java │ ├── MessageOutputStream.java │ ├── MessageStreamApi.java │ └── SnapshotManagementApi.java │ ├── impl │ ├── AdminImpl.java │ ├── AsyncImpl.java │ ├── BlockingImpl.java │ ├── ClientImplUtils.java │ ├── ClientProtoUtils.java │ ├── DataStreamClientImpl.java │ ├── GroupManagementImpl.java │ ├── LeaderElectionManagementImpl.java │ ├── MessageStreamImpl.java │ ├── OrderedAsync.java │ ├── OrderedStreamAsync.java │ ├── RaftClientImpl.java │ ├── RaftClientRpcWithProxy.java │ ├── RaftOutputStream.java │ ├── SnapshotManagementImpl.java │ └── UnorderedAsync.java │ └── retry │ ├── ClientRetryEvent.java │ └── RequestTypeDependentRetryPolicy.java ├── ratis-common ├── pom.xml └── src │ ├── main │ ├── conf │ │ └── log4j.properties │ └── java │ │ └── org │ │ └── apache │ │ └── ratis │ │ ├── RaftConfigKeys.java │ │ ├── conf │ │ ├── ConfUtils.java │ │ ├── Parameters.java │ │ ├── RaftProperties.java │ │ ├── Reconfigurable.java │ │ ├── ReconfigurationBase.java │ │ ├── ReconfigurationException.java │ │ └── ReconfigurationStatus.java │ │ ├── datastream │ │ ├── DataStreamFactory.java │ │ ├── DataStreamType.java │ │ ├── SupportedDataStreamType.java │ │ └── impl │ │ │ ├── DataStreamPacketByteBuffer.java │ │ │ ├── DataStreamPacketImpl.java │ │ │ ├── DataStreamReplyByteBuffer.java │ │ │ ├── DataStreamRequestByteBuf.java │ │ │ ├── DataStreamRequestByteBuffer.java │ │ │ └── DataStreamRequestFilePositionCount.java │ │ ├── io │ │ ├── CloseAsync.java │ │ ├── CorruptedFileException.java │ │ ├── FilePositionCount.java │ │ ├── MD5Hash.java │ │ ├── StandardWriteOption.java │ │ └── WriteOption.java │ │ ├── protocol │ │ ├── AdminAsynchronousProtocol.java │ │ ├── AdminProtocol.java │ │ ├── ClientId.java │ │ ├── ClientInvocationId.java │ │ ├── DataStreamPacket.java │ │ ├── DataStreamPacketHeader.java │ │ ├── DataStreamReply.java │ │ ├── DataStreamReplyHeader.java │ │ ├── DataStreamRequest.java │ │ ├── DataStreamRequestHeader.java │ │ ├── GroupInfoReply.java │ │ ├── GroupInfoRequest.java │ │ ├── GroupListReply.java │ │ ├── GroupListRequest.java │ │ ├── GroupManagementRequest.java │ │ ├── LeaderElectionManagementRequest.java │ │ ├── Message.java │ │ ├── RaftClientAsynchronousProtocol.java │ │ ├── RaftClientMessage.java │ │ ├── RaftClientProtocol.java │ │ ├── RaftClientReply.java │ │ ├── RaftClientRequest.java │ │ ├── RaftGroup.java │ │ ├── RaftGroupId.java │ │ ├── RaftGroupMemberId.java │ │ ├── RaftId.java │ │ ├── RaftPeer.java │ │ ├── RaftPeerId.java │ │ ├── RaftRpcMessage.java │ │ ├── RoutingTable.java │ │ ├── SetConfigurationRequest.java │ │ ├── SnapshotManagementRequest.java │ │ ├── TransferLeadershipRequest.java │ │ └── exceptions │ │ │ ├── AlreadyClosedException.java │ │ │ ├── AlreadyExistsException.java │ │ │ ├── ChecksumException.java │ │ │ ├── DataStreamException.java │ │ │ ├── GroupMismatchException.java │ │ │ ├── LeaderNotReadyException.java │ │ │ ├── LeaderSteppingDownException.java │ │ │ ├── NotLeaderException.java │ │ │ ├── NotReplicatedException.java │ │ │ ├── RaftException.java │ │ │ ├── RaftRetryFailureException.java │ │ │ ├── ReadException.java │ │ │ ├── ReadIndexException.java │ │ │ ├── ReconfigurationInProgressException.java │ │ │ ├── ReconfigurationTimeoutException.java │ │ │ ├── ResourceUnavailableException.java │ │ │ ├── ServerNotReadyException.java │ │ │ ├── SetConfigurationException.java │ │ │ ├── StaleReadException.java │ │ │ ├── StateMachineException.java │ │ │ ├── StreamException.java │ │ │ ├── TimeoutIOException.java │ │ │ └── TransferLeadershipException.java │ │ ├── retry │ │ ├── ExceptionDependentRetry.java │ │ ├── ExponentialBackoffRetry.java │ │ ├── MultipleLinearRandomRetry.java │ │ ├── RetryPolicies.java │ │ └── RetryPolicy.java │ │ ├── rpc │ │ ├── CallId.java │ │ ├── RpcFactory.java │ │ ├── RpcType.java │ │ └── SupportedRpcType.java │ │ ├── security │ │ └── TlsConf.java │ │ └── util │ │ ├── AtomicFileOutputStream.java │ │ ├── AutoCloseableLock.java │ │ ├── AutoCloseableReadWriteLock.java │ │ ├── AwaitForSignal.java │ │ ├── AwaitToRun.java │ │ ├── BatchLogger.java │ │ ├── BiWeakValueCache.java │ │ ├── CodeInjectionForTesting.java │ │ ├── CollectionUtils.java │ │ ├── ConcurrentUtils.java │ │ ├── Daemon.java │ │ ├── DataBlockingQueue.java │ │ ├── DataQueue.java │ │ ├── ExitUtils.java │ │ ├── FileUtils.java │ │ ├── IOUtils.java │ │ ├── JavaUtils.java │ │ ├── JmxRegister.java │ │ ├── JvmPauseMonitor.java │ │ ├── LifeCycle.java │ │ ├── LogUtils.java │ │ ├── LongMinMax.java │ │ ├── MD5FileUtil.java │ │ ├── MemoizedCheckedSupplier.java │ │ ├── MemoizedSupplier.java │ │ ├── NetUtils.java │ │ ├── OpenCloseState.java │ │ ├── PeerProxyMap.java │ │ ├── PlatformUtils.java │ │ ├── Preconditions.java │ │ ├── ProtoUtils.java │ │ ├── PureJavaCrc32C.java │ │ ├── RefCountingMap.java │ │ ├── ReferenceCountedObject.java │ │ ├── ReflectionUtils.java │ │ ├── ResourceSemaphore.java │ │ ├── SizeInBytes.java │ │ ├── SlidingWindow.java │ │ ├── StringUtils.java │ │ ├── TaskQueue.java │ │ ├── TimeDuration.java │ │ ├── TimeoutExecutor.java │ │ ├── TimeoutScheduler.java │ │ ├── TimeoutTimer.java │ │ ├── Timestamp.java │ │ ├── TraditionalBinaryPrefix.java │ │ ├── UncheckedAutoCloseable.java │ │ ├── WeakValueCache.java │ │ └── function │ │ ├── CheckedBiConsumer.java │ │ ├── CheckedBiFunction.java │ │ ├── CheckedConsumer.java │ │ ├── CheckedFunction.java │ │ ├── CheckedFunctionWithTimeout.java │ │ ├── CheckedRunnable.java │ │ ├── CheckedSupplier.java │ │ ├── FunctionUtils.java │ │ ├── StringSupplier.java │ │ ├── TriConsumer.java │ │ └── UncheckedAutoCloseableSupplier.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── ratis │ │ ├── BaseTest.java │ │ ├── JUnit5TestExecutionListener.java │ │ ├── test │ │ └── tag │ │ │ ├── Flaky.java │ │ │ └── FlakyTest.java │ │ └── util │ │ ├── Slf4jUtils.java │ │ ├── TestFileUtils.java │ │ ├── TestNetUtils.java │ │ ├── TestPureJavaCrc32C.java │ │ ├── TestRefCountingMap.java │ │ └── TestTaskQueue.java │ └── resources │ ├── META-INF │ └── services │ │ └── org.junit.platform.launcher.TestExecutionListener │ └── log4j.properties ├── ratis-docs ├── README.md ├── pom.xml └── src │ └── site │ ├── markdown │ ├── cli.md │ ├── concept │ │ └── index.md │ ├── configurations.md │ ├── index.md │ ├── membership-change.md │ ├── metrics.md │ ├── security.md │ ├── snapshot.md │ └── start │ │ └── index.md │ ├── resources │ └── images │ │ ├── asf_logo.svg │ │ ├── feather.svg │ │ └── ratis.png │ └── site.xml ├── ratis-examples ├── README.md ├── pom.xml └── src │ ├── main │ ├── bin │ │ ├── client.sh │ │ ├── common.sh │ │ ├── server.sh │ │ ├── start-all.sh │ │ └── stop-all.sh │ ├── docker │ │ ├── Dockerfile │ │ └── docker-compose.yaml │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── ratis │ │ │ └── examples │ │ │ ├── arithmetic │ │ │ ├── ArithmeticLogDump.java │ │ │ ├── ArithmeticStateMachine.java │ │ │ ├── AssignmentMessage.java │ │ │ ├── Evaluable.java │ │ │ ├── cli │ │ │ │ ├── Arithmetic.java │ │ │ │ ├── Assign.java │ │ │ │ ├── Client.java │ │ │ │ ├── Get.java │ │ │ │ └── Server.java │ │ │ └── expression │ │ │ │ ├── BinaryExpression.java │ │ │ │ ├── DoubleValue.java │ │ │ │ ├── Expression.java │ │ │ │ ├── NullValue.java │ │ │ │ ├── UnaryExpression.java │ │ │ │ └── Variable.java │ │ │ ├── common │ │ │ ├── Constants.java │ │ │ ├── Runner.java │ │ │ └── SubCommandBase.java │ │ │ ├── counter │ │ │ ├── CounterCommand.java │ │ │ ├── client │ │ │ │ └── CounterClient.java │ │ │ └── server │ │ │ │ ├── CounterServer.java │ │ │ │ └── CounterStateMachine.java │ │ │ ├── debug │ │ │ └── server │ │ │ │ └── Server.java │ │ │ ├── filestore │ │ │ ├── FileInfo.java │ │ │ ├── FileStore.java │ │ │ ├── FileStoreClient.java │ │ │ ├── FileStoreCommon.java │ │ │ ├── FileStoreStateMachine.java │ │ │ └── cli │ │ │ │ ├── Client.java │ │ │ │ ├── DataStream.java │ │ │ │ ├── FileStore.java │ │ │ │ ├── LoadGen.java │ │ │ │ └── Server.java │ │ │ └── membership │ │ │ └── server │ │ │ ├── CServer.java │ │ │ ├── Console.java │ │ │ └── RaftCluster.java │ └── resources │ │ ├── conf.properties │ │ └── log4j.properties │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── ratis │ │ ├── TestMultiRaftGroup.java │ │ ├── examples │ │ ├── ParameterizedBaseTest.java │ │ ├── arithmetic │ │ │ ├── TestArithmetic.java │ │ │ ├── TestArithmeticLogDump.java │ │ │ ├── cli │ │ │ │ └── TestAssignCli.java │ │ │ └── expression │ │ │ │ └── TestExpression.java │ │ ├── common │ │ │ └── TestSubCommand.java │ │ ├── counter │ │ │ └── TestCounter.java │ │ └── filestore │ │ │ ├── FileStoreAsyncBaseTest.java │ │ │ ├── FileStoreBaseTest.java │ │ │ ├── FileStoreStreamingBaseTest.java │ │ │ ├── FileStoreWriter.java │ │ │ ├── TestFileStoreAsyncWithGrpc.java │ │ │ ├── TestFileStoreStreamingWithGrpcCluster.java │ │ │ ├── TestFileStoreWithGrpc.java │ │ │ └── TestFileStoreWithNetty.java │ │ └── server │ │ └── impl │ │ └── TestReadAfterWrite.java │ └── resources │ └── log4j.properties ├── ratis-experiments ├── README.md ├── pom.xml ├── scripts │ └── flatbuf-compile.sh └── src │ └── main │ ├── flatbufs │ └── FileTransfer.fbs │ └── java │ └── org │ └── apache │ └── ratis │ └── experiments │ ├── flatbuffers │ ├── client │ │ ├── ClientFlat.java │ │ └── ClientProto.java │ └── server │ │ ├── FileTransferFlatbufs.java │ │ ├── FileTransferProtobufs.java │ │ ├── ServerFlat.java │ │ └── ServerProto.java │ └── nettyzerocopy │ ├── client │ └── NettyClient.java │ ├── decoders │ ├── RequestDecoder.java │ ├── RequestDecoderComposite.java │ └── ResponseDecoder.java │ ├── encoders │ ├── RequestEncoder.java │ └── ResponseEncoder.java │ ├── objects │ ├── RequestData.java │ ├── RequestDataComposite.java │ └── ResponseData.java │ └── server │ └── NettyServer.java ├── ratis-grpc ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── ratis │ │ └── grpc │ │ ├── GrpcConfigKeys.java │ │ ├── GrpcFactory.java │ │ ├── GrpcTlsConfig.java │ │ ├── GrpcUtil.java │ │ ├── client │ │ ├── GrpcClientProtocolClient.java │ │ ├── GrpcClientProtocolProxy.java │ │ └── GrpcClientRpc.java │ │ ├── metrics │ │ ├── GrpcServerMetrics.java │ │ ├── MessageMetrics.java │ │ └── intercept │ │ │ ├── client │ │ │ ├── MetricClientCall.java │ │ │ ├── MetricClientCallListener.java │ │ │ └── MetricClientInterceptor.java │ │ │ └── server │ │ │ ├── MetricServerCall.java │ │ │ └── MetricServerInterceptor.java │ │ ├── server │ │ ├── GrpcAdminProtocolService.java │ │ ├── GrpcClientProtocolService.java │ │ ├── GrpcLogAppender.java │ │ ├── GrpcServerProtocolClient.java │ │ ├── GrpcServerProtocolService.java │ │ ├── GrpcServices.java │ │ └── GrpcServicesImpl.java │ │ └── util │ │ ├── ResponseNotifyClientInterceptor.java │ │ ├── StreamObserverWithTimeout.java │ │ └── ZeroCopyMessageMarshaller.java │ └── test │ └── java │ └── org │ └── apache │ └── ratis │ └── grpc │ └── MiniRaftClusterWithGrpc.java ├── ratis-metrics-api ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── ratis │ └── metrics │ ├── LongCounter.java │ ├── MetricRegistries.java │ ├── MetricRegistriesLoader.java │ ├── MetricRegistryFactory.java │ ├── MetricRegistryInfo.java │ ├── RatisMetricRegistry.java │ ├── RatisMetrics.java │ └── Timekeeper.java ├── ratis-metrics-default ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── ratis │ │ │ └── metrics │ │ │ └── impl │ │ │ ├── DefaultTimekeeperImpl.java │ │ │ ├── JvmMetrics.java │ │ │ ├── MetricRegistriesImpl.java │ │ │ ├── MetricRegistryFactoryImpl.java │ │ │ ├── MetricsReporting.java │ │ │ ├── RatisMetricRegistryImpl.java │ │ │ └── RatisObjectNameFactory.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.ratis.metrics.MetricRegistries │ └── test │ └── java │ └── org │ └── apache │ └── ratis │ └── metrics │ └── TestMetricRegistriesLoader.java ├── ratis-metrics-dropwizard3 ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── ratis │ │ │ └── metrics │ │ │ └── dropwizard3 │ │ │ ├── Dm3JvmMetrics.java │ │ │ ├── Dm3MetricRegistriesImpl.java │ │ │ ├── Dm3MetricRegistryFactoryImpl.java │ │ │ ├── Dm3MetricsReporting.java │ │ │ ├── Dm3RatisMetricRegistryImpl.java │ │ │ ├── Dm3RatisObjectNameFactory.java │ │ │ └── Dm3TimekeeperImpl.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.ratis.metrics.MetricRegistries │ └── test │ └── java │ └── org │ └── apache │ └── ratis │ └── metrics │ └── dropwizard3 │ └── TestLoadDm3MetricRegistries.java ├── ratis-netty ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── ratis │ │ └── netty │ │ ├── NettyClient.java │ │ ├── NettyConfigKeys.java │ │ ├── NettyDataStreamFactory.java │ │ ├── NettyDataStreamUtils.java │ │ ├── NettyFactory.java │ │ ├── NettyRpcProxy.java │ │ ├── NettyUtils.java │ │ ├── client │ │ ├── NettyClientReplies.java │ │ ├── NettyClientRpc.java │ │ └── NettyClientStreamRpc.java │ │ ├── metrics │ │ └── NettyServerStreamRpcMetrics.java │ │ └── server │ │ ├── ChannelMap.java │ │ ├── DataStreamManagement.java │ │ ├── NettyRpcService.java │ │ ├── NettyServerStreamRpc.java │ │ └── StreamMap.java │ └── test │ └── java │ └── org │ └── apache │ └── ratis │ └── netty │ └── MiniRaftClusterWithNetty.java ├── ratis-proto ├── pom.xml └── src │ └── main │ └── proto │ ├── Examples.proto │ ├── Experiments.proto │ ├── Grpc.proto │ ├── Netty.proto │ ├── RMap.proto │ ├── Raft.proto │ └── Test.proto ├── ratis-replicated-map ├── README.md ├── pom.xml └── src │ └── test │ └── resources │ └── log4j.properties ├── ratis-resource-bundle ├── pom.xml └── src │ └── main │ └── resources │ ├── META-INF │ ├── LICENSE.vm │ └── NOTICE.vm │ └── supplemental-models.xml ├── ratis-server-api ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── ratis │ ├── server │ ├── DataStreamMap.java │ ├── DataStreamServer.java │ ├── DataStreamServerFactory.java │ ├── DataStreamServerRpc.java │ ├── DivisionInfo.java │ ├── DivisionProperties.java │ ├── RaftConfiguration.java │ ├── RaftServer.java │ ├── RaftServerConfigKeys.java │ ├── RaftServerMXBean.java │ ├── RaftServerRpc.java │ ├── RaftServerRpcWithProxy.java │ ├── RetryCache.java │ ├── ServerFactory.java │ ├── ServerRpc.java │ ├── leader │ │ ├── FollowerInfo.java │ │ ├── LeaderState.java │ │ └── LogAppender.java │ ├── metrics │ │ ├── RaftLogMetrics.java │ │ └── RaftServerMetrics.java │ ├── protocol │ │ ├── RaftServerAsynchronousProtocol.java │ │ ├── RaftServerProtocol.java │ │ └── TermIndex.java │ ├── raftlog │ │ ├── LogEntryHeader.java │ │ ├── RaftLog.java │ │ ├── RaftLogIOException.java │ │ ├── RaftLogIndex.java │ │ └── RaftLogSequentialOps.java │ └── storage │ │ ├── FileInfo.java │ │ ├── RaftStorage.java │ │ ├── RaftStorageDirectory.java │ │ ├── RaftStorageMetadata.java │ │ └── RaftStorageMetadataFile.java │ └── statemachine │ ├── SnapshotInfo.java │ ├── SnapshotRetentionPolicy.java │ ├── StateMachine.java │ ├── StateMachineStorage.java │ └── TransactionContext.java ├── ratis-server ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── ratis │ │ ├── server │ │ ├── DisabledDataStreamServerFactory.java │ │ ├── impl │ │ │ ├── CommitInfoCache.java │ │ │ ├── ConfigurationManager.java │ │ │ ├── DataStreamMapImpl.java │ │ │ ├── DataStreamServerImpl.java │ │ │ ├── DivisionPropertiesImpl.java │ │ │ ├── FollowerInfoImpl.java │ │ │ ├── FollowerState.java │ │ │ ├── LeaderElection.java │ │ │ ├── LeaderLease.java │ │ │ ├── LeaderStateImpl.java │ │ │ ├── MessageStreamRequests.java │ │ │ ├── PeerConfiguration.java │ │ │ ├── PendingRequest.java │ │ │ ├── PendingRequests.java │ │ │ ├── PendingStepDown.java │ │ │ ├── RaftConfigurationImpl.java │ │ │ ├── RaftServerImpl.java │ │ │ ├── RaftServerJmxAdapter.java │ │ │ ├── RaftServerProxy.java │ │ │ ├── ReadIndexHeartbeats.java │ │ │ ├── ReadRequests.java │ │ │ ├── RetryCacheImpl.java │ │ │ ├── RoleInfo.java │ │ │ ├── ServerImplUtils.java │ │ │ ├── ServerProtoUtils.java │ │ │ ├── ServerState.java │ │ │ ├── SnapshotInstallationHandler.java │ │ │ ├── SnapshotManagementRequestHandler.java │ │ │ ├── StateMachineMetrics.java │ │ │ ├── StateMachineUpdater.java │ │ │ ├── TransactionManager.java │ │ │ ├── TransferLeadership.java │ │ │ ├── VoteContext.java │ │ │ ├── WatchRequests.java │ │ │ └── WriteIndexCache.java │ │ ├── leader │ │ │ ├── InstallSnapshotRequests.java │ │ │ ├── LeaderProtoUtils.java │ │ │ ├── LogAppenderBase.java │ │ │ ├── LogAppenderDaemon.java │ │ │ └── LogAppenderDefault.java │ │ ├── metrics │ │ │ ├── LeaderElectionMetrics.java │ │ │ ├── LogAppenderMetrics.java │ │ │ ├── RaftLogMetricsBase.java │ │ │ ├── RaftServerMetricsImpl.java │ │ │ └── SegmentedRaftLogMetrics.java │ │ ├── raftlog │ │ │ ├── LogProtoUtils.java │ │ │ ├── RaftLogBase.java │ │ │ ├── memory │ │ │ │ └── MemoryRaftLog.java │ │ │ └── segmented │ │ │ │ ├── BufferedWriteChannel.java │ │ │ │ ├── CacheInvalidationPolicy.java │ │ │ │ ├── LogSegment.java │ │ │ │ ├── LogSegmentPath.java │ │ │ │ ├── LogSegmentStartEnd.java │ │ │ │ ├── SegmentedRaftLog.java │ │ │ │ ├── SegmentedRaftLogCache.java │ │ │ │ ├── SegmentedRaftLogFormat.java │ │ │ │ ├── SegmentedRaftLogInputStream.java │ │ │ │ ├── SegmentedRaftLogOutputStream.java │ │ │ │ ├── SegmentedRaftLogReader.java │ │ │ │ └── SegmentedRaftLogWorker.java │ │ ├── storage │ │ │ ├── FileChunkReader.java │ │ │ ├── RaftStorageDirectoryImpl.java │ │ │ ├── RaftStorageImpl.java │ │ │ ├── RaftStorageMetadataFileImpl.java │ │ │ ├── SnapshotManager.java │ │ │ └── StorageImplUtils.java │ │ └── util │ │ │ └── ServerStringUtils.java │ │ └── statemachine │ │ └── impl │ │ ├── BaseStateMachine.java │ │ ├── FileListSnapshotInfo.java │ │ ├── SimpleStateMachineStorage.java │ │ ├── SingleFileSnapshotInfo.java │ │ └── TransactionContextImpl.java │ └── test │ └── java │ └── org │ └── apache │ └── ratis │ ├── InstallSnapshotFromLeaderTests.java │ ├── InstallSnapshotNotificationTests.java │ ├── LogAppenderTests.java │ ├── MessageStreamApiTests.java │ ├── OutputStreamBaseTest.java │ ├── RaftAsyncExceptionTests.java │ ├── RaftAsyncTests.java │ ├── RaftBasicTests.java │ ├── RaftExceptionBaseTest.java │ ├── RaftTestUtil.java │ ├── ReadOnlyRequestTests.java │ ├── ReadOnlyRequestWithLongTimeoutTests.java │ ├── RequestLimitAsyncBaseTest.java │ ├── RetryCacheTests.java │ ├── TestReConfigProperty.java │ ├── WatchRequestTests.java │ ├── client │ └── impl │ │ └── RaftClientTestUtil.java │ ├── server │ ├── impl │ │ ├── BlockRequestHandlingInjection.java │ │ ├── DelayLocalExecutionInjection.java │ │ ├── GroupInfoBaseTest.java │ │ ├── GroupManagementBaseTest.java │ │ ├── LeaderElectionTests.java │ │ ├── LogMetadataTests.java │ │ ├── MiniRaftCluster.java │ │ ├── PreAppendLeaderStepDownTest.java │ │ ├── RaftReconfigurationBaseTest.java │ │ ├── RaftServerTestUtil.java │ │ ├── RaftStateMachineExceptionTests.java │ │ ├── RetryCacheTestUtil.java │ │ ├── ServerPauseResumeTest.java │ │ ├── StateMachineShutdownTests.java │ │ ├── TestLogAppenderMetrics.java │ │ ├── TestRatisServerMetricsBase.java │ │ └── TestRetryCacheMetrics.java │ ├── metrics │ │ ├── ServerMetricsTestUtils.java │ │ └── TestLeaderElectionMetrics.java │ ├── raftlog │ │ └── segmented │ │ │ └── SegmentedRaftLogTestUtils.java │ ├── simulation │ │ ├── MiniRaftClusterWithSimulatedRpc.java │ │ ├── RaftServerReply.java │ │ ├── RaftServerRequest.java │ │ ├── RequestHandler.java │ │ ├── SimulatedClientRpc.java │ │ ├── SimulatedRequestReply.java │ │ ├── SimulatedRpc.java │ │ └── SimulatedServerRpc.java │ └── storage │ │ └── RaftStorageTestUtils.java │ └── statemachine │ ├── RaftSnapshotBaseTest.java │ ├── SnapshotManagementTest.java │ └── impl │ └── SimpleStateMachine4Testing.java ├── ratis-shell ├── pom.xml └── src │ └── main │ ├── bin │ └── ratis │ ├── conf │ └── log4j.properties │ ├── java │ └── org │ │ └── apache │ │ └── ratis │ │ └── shell │ │ └── cli │ │ ├── AbstractShell.java │ │ ├── CliUtils.java │ │ ├── Command.java │ │ └── sh │ │ ├── RatisShell.java │ │ ├── command │ │ ├── AbstractCommand.java │ │ ├── AbstractParentCommand.java │ │ ├── AbstractRatisCommand.java │ │ ├── Context.java │ │ ├── ElectionCommand.java │ │ ├── GroupCommand.java │ │ ├── LocalCommand.java │ │ ├── PeerCommand.java │ │ └── SnapshotCommand.java │ │ ├── election │ │ ├── PauseCommand.java │ │ ├── ResumeCommand.java │ │ ├── StepDownCommand.java │ │ └── TransferCommand.java │ │ ├── group │ │ ├── GroupInfoCommand.java │ │ └── GroupListCommand.java │ │ ├── local │ │ └── RaftMetaConfCommand.java │ │ ├── peer │ │ ├── AddCommand.java │ │ ├── RemoveCommand.java │ │ └── SetPriorityCommand.java │ │ └── snapshot │ │ └── TakeSnapshotCommand.java │ └── libexec │ └── ratis-shell-config.sh ├── ratis-test ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── ratis │ │ ├── RaftLogTruncateTests.java │ │ ├── TestRaftServerNoLeaderTimeout.java │ │ ├── TestRaftServerSlownessDetection.java │ │ ├── client │ │ └── TestClientProtoUtils.java │ │ ├── conf │ │ ├── TestConfUtils.java │ │ └── TestRaftProperties.java │ │ ├── datastream │ │ ├── DataStreamAsyncClusterTests.java │ │ ├── DataStreamBaseTest.java │ │ ├── DataStreamClusterTests.java │ │ ├── DataStreamTestUtils.java │ │ ├── MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.java │ │ ├── MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.java │ │ ├── TestDataStreamDisabled.java │ │ ├── TestDataStreamSslWithRpcTypeGrpcAndDataStreamTypeNetty.java │ │ ├── TestNettyDataStreamChainTopologyWithGrpcCluster.java │ │ ├── TestNettyDataStreamStarTopologyWithGrpcCluster.java │ │ ├── TestNettyDataStreamWithMock.java │ │ └── TestNettyDataStreamWithNettyCluster.java │ │ ├── grpc │ │ ├── TestCustomGrpcServices.java │ │ ├── TestGroupInfoWithGrpc.java │ │ ├── TestGroupManagementWithGrpc.java │ │ ├── TestGrpcFactory.java │ │ ├── TestInstallSnapshotNotificationWithGrpc.java │ │ ├── TestLeaderElectionWithGrpc.java │ │ ├── TestLeaderInstallSnapshotWithGrpc.java │ │ ├── TestLogAppenderWithGrpc.java │ │ ├── TestLogMetadataTestsWithGrpc.java │ │ ├── TestMessageStreamApiWithGrpc.java │ │ ├── TestPreAppendLeaderStepDownWithGrpc.java │ │ ├── TestRaftAsyncExceptionWithGrpc.java │ │ ├── TestRaftAsyncWithGrpc.java │ │ ├── TestRaftExceptionWithGrpc.java │ │ ├── TestRaftLogTruncateWithGrpc.java │ │ ├── TestRaftOutputStreamWithGrpc.java │ │ ├── TestRaftReconfigurationWithGrpc.java │ │ ├── TestRaftServerWithGrpc.java │ │ ├── TestRaftSnapshotWithGrpc.java │ │ ├── TestRaftStateMachineExceptionWithGrpc.java │ │ ├── TestRaftWithGrpc.java │ │ ├── TestReConfigPropertyWithGrpc.java │ │ ├── TestReadOnlyRequestWithGrpc.java │ │ ├── TestReadOnlyRequestWithLongTimeoutWithGrpc.java │ │ ├── TestRequestLimitAsyncWithGrpc.java │ │ ├── TestRetryCacheWithGrpc.java │ │ ├── TestServerPauseResumeWithGrpc.java │ │ ├── TestServerRestartWithGrpc.java │ │ ├── TestSnapshotManagementWithGrpc.java │ │ ├── TestStateMachineShutdownWithGrpc.java │ │ ├── TestWatchRequestWithGrpc.java │ │ ├── cli │ │ │ ├── TestElectionCommandIntegrationWithGrpc.java │ │ │ ├── TestGroupCommandIntegrationWithGrpc.java │ │ │ ├── TestPeerCommandIntegrationWithGrpc.java │ │ │ └── TestSnapshotCommandIntegrationWithGrpc.java │ │ ├── server │ │ │ ├── TestGrpcMessageMetrics.java │ │ │ └── TestGrpcServerMetrics.java │ │ └── util │ │ │ ├── GrpcTestClient.java │ │ │ ├── GrpcTestServer.java │ │ │ ├── GrpcZeroCopyTestClient.java │ │ │ ├── GrpcZeroCopyTestServer.java │ │ │ ├── TestGrpcZeroCopy.java │ │ │ └── TestStreamObserverWithTimeout.java │ │ ├── netty │ │ ├── TestGroupInfoWithNetty.java │ │ ├── TestGroupManagementWithNetty.java │ │ ├── TestLeaderElectionWithNetty.java │ │ ├── TestLogAppenderWithNetty.java │ │ ├── TestPreAppendLeaderStepDownWithNetty.java │ │ ├── TestRaftAsyncWithNetty.java │ │ ├── TestRaftExceptionWithNetty.java │ │ ├── TestRaftReconfigurationWithNetty.java │ │ ├── TestRaftSnapshotWithNetty.java │ │ ├── TestRaftStateMachineExceptionWithNetty.java │ │ ├── TestRaftWithNetty.java │ │ ├── TestRetryCacheWithNettyRpc.java │ │ ├── TestServerPauseResumeWithNetty.java │ │ ├── TestServerRestartWithNetty.java │ │ ├── TestSnapshotManagementWithNetty.java │ │ └── TestTlsConfWithNetty.java │ │ ├── protocol │ │ ├── TestRaftGroup.java │ │ ├── TestRaftId.java │ │ └── TestRoutingTable.java │ │ ├── retry │ │ ├── TestExceptionDependentRetry.java │ │ ├── TestExponentialBackoffRetry.java │ │ ├── TestMultipleLinearRandomRetry.java │ │ └── TestRetryPolicy.java │ │ ├── security │ │ └── SecurityTestUtils.java │ │ ├── server │ │ ├── ServerBuilderTest.java │ │ ├── ServerRestartTests.java │ │ ├── TestRaftServerConfigKeys.java │ │ ├── impl │ │ │ ├── TestLeaderElectionServerInterface.java │ │ │ ├── TestPeerConfiguration.java │ │ │ ├── TestRaftConfiguration.java │ │ │ └── TestRaftServerJmx.java │ │ ├── protocol │ │ │ └── ProtocolTestUtils.java │ │ ├── raftlog │ │ │ ├── TestRaftLogIndex.java │ │ │ ├── TestRaftLogMetrics.java │ │ │ ├── memory │ │ │ │ └── MemoryRaftLogTest.java │ │ │ └── segmented │ │ │ │ ├── TestBufferedWriteChannel.java │ │ │ │ ├── TestCacheEviction.java │ │ │ │ ├── TestLogSegment.java │ │ │ │ ├── TestRaftLogReadWrite.java │ │ │ │ ├── TestSegmentedRaftLog.java │ │ │ │ └── TestSegmentedRaftLogCache.java │ │ ├── simulation │ │ │ ├── TestGroupInfoWithSimulatedRpc.java │ │ │ ├── TestGroupManagementWithSimulatedRpc.java │ │ │ ├── TestLeaderElectionWithSimulatedRpc.java │ │ │ ├── TestLogAppenderWithSimulatedRpc.java │ │ │ ├── TestPreAppendLeaderStepDownWithSimulatedRpc.java │ │ │ ├── TestRaftExceptionWithSimulation.java │ │ │ ├── TestRaftReconfigurationWithSimulatedRpc.java │ │ │ ├── TestRaftSnapshotWithSimulatedRpc.java │ │ │ ├── TestRaftStateMachineExceptionWithSimulatedRpc.java │ │ │ ├── TestRaftWithSimulatedRpc.java │ │ │ ├── TestRatisServerMetricsWithSimulatedRpc.java │ │ │ ├── TestRetryCacheWithSimulatedRpc.java │ │ │ ├── TestServerPauseResumeWithSimulatedRpc.java │ │ │ ├── TestServerRestartWithSimulatedRpc.java │ │ │ └── TestSnapshotManagementWithSimulatedRpc.java │ │ └── storage │ │ │ ├── TestRaftStorage.java │ │ │ └── TestStorageImplUtils.java │ │ ├── shell │ │ └── cli │ │ │ └── sh │ │ │ ├── AbstractCommandIntegrationTestWithGrpc.java │ │ │ ├── ElectionCommandIntegrationTest.java │ │ │ ├── GroupCommandIntegrationTest.java │ │ │ ├── LocalCommandIntegrationTest.java │ │ │ ├── PeerCommandIntegrationTest.java │ │ │ ├── SnapshotCommandIntegrationTest.java │ │ │ ├── StringPrintStream.java │ │ │ ├── TestRatisShell.java │ │ │ └── TestSecureRatisShell.java │ │ ├── statemachine │ │ └── TestStateMachine.java │ │ └── util │ │ ├── TestDataBlockingQueue.java │ │ ├── TestDataQueue.java │ │ ├── TestExitUtils.java │ │ ├── TestLifeCycle.java │ │ ├── TestMinMax.java │ │ ├── TestPeerProxyMap.java │ │ ├── TestPreconditions.java │ │ ├── TestRaftIdCache.java │ │ ├── TestReferenceCountedObject.java │ │ ├── TestResourceSemaphore.java │ │ ├── TestTermIndex.java │ │ ├── TestTimeDuration.java │ │ ├── TestTimeoutScheduler.java │ │ └── TestTraditionalBinaryPrefix.java │ └── resources │ ├── log4j.properties │ ├── mockito-extensions │ └── org.mockito.plugins.MockMaker │ └── ssl │ ├── ca.crt │ ├── ca.key │ ├── client.crt │ ├── client.csr │ ├── client.key │ ├── client.pem │ ├── generate.sh │ ├── server.crt │ ├── server.csr │ ├── server.key │ └── server.pem ├── ratis-tools ├── pom.xml └── src │ └── main │ └── java │ └── org │ └── apache │ └── ratis │ └── tools │ ├── DefaultLogDump.java │ └── ParseRatisLog.java ├── src └── main │ └── resources │ └── ratis-version.properties └── start-build-env.sh /.asf.yaml: -------------------------------------------------------------------------------- 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 | github: 16 | description: "Open source Java implementation for Raft consensus protocol." 17 | homepage: https://ratis.apache.org/ 18 | labels: 19 | - raft 20 | - consensus-protocol 21 | - consensus 22 | - java 23 | enabled_merge_buttons: 24 | squash: true 25 | squash_commit_message: PR_TITLE 26 | merge: false 27 | rebase: false 28 | autolink_jira: 29 | - HDDS 30 | - RATIS 31 | 32 | notifications: 33 | commits: commits@ratis.apache.org 34 | issues: issues@ratis.apache.org 35 | pullrequests: issues@ratis.apache.org 36 | jira_options: link label worklog 37 | 38 | publish: 39 | whoami: asf-site 40 | -------------------------------------------------------------------------------- /.github/pull_request_template.md: -------------------------------------------------------------------------------- 1 | ## What changes were proposed in this pull request? 2 | 3 | (Please fill in changes proposed in this fix) 4 | 5 | ## What is the link to the Apache JIRA 6 | 7 | (Please create an issue in ASF JIRA before opening a pull request, 8 | and you need to set the title of the pull request which starts with 9 | the corresponding JIRA issue number. (e.g. RATIS-XXXX. Fix a typo in YYY.) 10 | 11 | Please replace this section with the link to the Apache JIRA) 12 | 13 | ## How was this patch tested? 14 | 15 | (Please explain how this patch was tested. Ex: unit tests, manual tests) 16 | (If this patch involves UI changes, please attach a screen-shot; otherwise, remove this) 17 | -------------------------------------------------------------------------------- /.github/workflows/post-commit.yaml: -------------------------------------------------------------------------------- 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 | name: build-branch 17 | 18 | on: 19 | push: 20 | branches-ignore: 21 | - 'dependabot/**' 22 | tags: 23 | - '**' 24 | pull_request: 25 | 26 | concurrency: 27 | group: ci-${{ github.event.pull_request.number || github.sha }} 28 | cancel-in-progress: ${{ github.event_name == 'pull_request' }} 29 | 30 | jobs: 31 | CI: 32 | uses: ./.github/workflows/ci.yaml 33 | secrets: inherit 34 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .DS_Store 2 | *.iml 3 | *.ipr 4 | *.iws 5 | *.orig 6 | *.rej 7 | **/.keep 8 | *.sdf 9 | *.suo 10 | *.vcxproj.user 11 | .hugo_build.lock 12 | .idea 13 | .classpath 14 | .mvn/.develocity/ 15 | .project 16 | .settings 17 | target 18 | build 19 | patchprocess 20 | dependency-reduced-pom.xml 21 | .mvn/wrapper/maven-wrapper.jar 22 | .vscode/ 23 | -------------------------------------------------------------------------------- /.idea/vcs.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 19 | 20 | -------------------------------------------------------------------------------- /.mvn/extensions.xml: -------------------------------------------------------------------------------- 1 | 2 | 22 | 24 | 25 | com.gradle 26 | develocity-maven-extension 27 | 1.22.2 28 | 29 | 30 | com.gradle 31 | common-custom-user-data-maven-extension 32 | 2.0.1 33 | 34 | 35 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | wrapperVersion=3.3.2 18 | distributionType=only-script 19 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.9/apache-maven-3.9.9-bin.zip 20 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Ratis 2 | Copyright 2017-2023 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -- 7 | 8 | -------------------------------------------------------------------------------- /dev-support/checks/_summary.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | REPORT_FILE="$1" 18 | 19 | : ${ITERATIONS:="1"} 20 | 21 | declare -i ITERATIONS 22 | 23 | rc=0 24 | 25 | if [[ ! -e "${REPORT_FILE}" ]]; then 26 | echo "Report file missing, check logs for details" 27 | rc=255 28 | 29 | elif [[ ${ITERATIONS} -gt 1 ]]; then 30 | cat "${REPORT_FILE}" 31 | 32 | if grep -q 'exit code: [^0]' "${REPORT_FILE}"; then 33 | rc=1 34 | fi 35 | 36 | elif [[ -s "${REPORT_FILE}" ]]; then 37 | cat "${REPORT_FILE}" 38 | rc=1 39 | fi 40 | 41 | exit ${rc} 42 | -------------------------------------------------------------------------------- /dev-support/checks/author.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 17 | cd "$DIR/../.." || exit 1 18 | 19 | REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/author"} 20 | mkdir -p "$REPORT_DIR" 21 | REPORT_FILE="$REPORT_DIR/summary.txt" 22 | 23 | grep -r --include="*.java" "@author" . | tee "$REPORT_FILE" 24 | 25 | wc -l "$REPORT_FILE" | awk '{print $1}'> "$REPORT_DIR/failures" 26 | 27 | if [[ -s "${REPORT_FILE}" ]]; then 28 | exit 1 29 | fi 30 | -------------------------------------------------------------------------------- /dev-support/checks/build.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 17 | cd "$DIR/../.." || exit 1 18 | 19 | source "${DIR}/../find_maven.sh" 20 | 21 | : ${WITH_COVERAGE:="false"} 22 | 23 | MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' 24 | 25 | if [[ "${WITH_COVERAGE}" != "true" ]]; then 26 | MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" 27 | fi 28 | 29 | export MAVEN_OPTS="-Xmx4096m" 30 | ${MVN} ${MAVEN_OPTIONS} clean install "$@" 31 | exit $? 32 | -------------------------------------------------------------------------------- /dev-support/checks/compile.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 17 | cd "$DIR/../.." || exit 1 18 | 19 | source "${DIR}/../find_maven.sh" 20 | 21 | : ${WITH_COVERAGE:="false"} 22 | 23 | MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' 24 | 25 | if [[ "${WITH_COVERAGE}" != "true" ]]; then 26 | MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" 27 | fi 28 | 29 | export MAVEN_OPTS="-Xmx4096m" 30 | ${MVN} ${MAVEN_OPTIONS} clean verify "$@" 31 | exit $? 32 | -------------------------------------------------------------------------------- /dev-support/checks/rat.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 17 | cd "$DIR/../.." || exit 1 18 | 19 | source "${DIR}/../find_maven.sh" 20 | 21 | REPORT_DIR=${OUTPUT_DIR:-"$DIR/../../target/rat"} 22 | mkdir -p "$REPORT_DIR" 23 | 24 | REPORT_FILE="$REPORT_DIR/summary.txt" 25 | 26 | ${MVN} -B -fn org.apache.rat:apache-rat-plugin:0.13:check 27 | 28 | cd "$DIR/../.." || exit 1 29 | 30 | grep -r --include=rat.txt "!????" . | tee "$REPORT_FILE" 31 | 32 | wc -l "$REPORT_FILE" | awk '{print $1}'> "$REPORT_DIR/failures" 33 | 34 | if [[ -s "${REPORT_FILE}" ]]; then 35 | exit 1 36 | fi 37 | 38 | -------------------------------------------------------------------------------- /dev-support/checks/repro.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 17 | cd "$DIR/../.." || exit 1 18 | 19 | source "${DIR}/../find_maven.sh" 20 | 21 | : ${WITH_COVERAGE:="false"} 22 | 23 | MAVEN_OPTIONS='-V -B -Dmaven.javadoc.skip=true -DskipTests' 24 | 25 | if [[ "${WITH_COVERAGE}" != "true" ]]; then 26 | MAVEN_OPTIONS="${MAVEN_OPTIONS} -Djacoco.skip" 27 | fi 28 | 29 | export MAVEN_OPTS="-Xmx4096m" 30 | ${MVN} ${MAVEN_OPTIONS} clean verify artifact:compare "$@" 31 | exit $? 32 | -------------------------------------------------------------------------------- /dev-support/checks/sonar.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)" 17 | cd "$DIR/../.." || exit 1 18 | 19 | source "${DIR}/../find_maven.sh" 20 | 21 | if [ ! "$SONAR_TOKEN" ]; then 22 | echo "SONAR_TOKEN environment variable should be set" 23 | exit 1 24 | fi 25 | 26 | ${MVN} -B verify -DskipShade -DskipTests \ 27 | sonar:sonar \ 28 | -Dsonar.coverage.jacoco.xmlReportPaths="$(pwd)/target/coverage/all.xml" \ 29 | -Dsonar.host.url=https://sonarcloud.io -Dsonar.organization=apache -Dsonar.projectKey=apache-ratis 30 | -------------------------------------------------------------------------------- /dev-support/find_maven.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | 19 | function find_maven() { 20 | if [ "$MAVEN" != "" ]; then 21 | echo "${MAVEN}" 22 | else 23 | local DIR 24 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 25 | ( cd "$DIR/.." || exit 1 ; echo "$( pwd -P )/mvnw" ) 26 | fi 27 | } 28 | 29 | MVN="$( find_maven )" 30 | export MVN 31 | -------------------------------------------------------------------------------- /dev-support/intellij/install-runconfig.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with 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 | SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 19 | SRC_DIR="$SCRIPT_DIR/runConfigurations" 20 | DEST_DIR="$SCRIPT_DIR/../../.idea/runConfigurations/" 21 | echo src : "${SRC_DIR}" 22 | echo dest: "${DEST_DIR}" 23 | 24 | set -ex 25 | mkdir -p "$DEST_DIR" 26 | cp -i "${SRC_DIR}"/* "${DEST_DIR}" -------------------------------------------------------------------------------- /dev-support/intellij/runConfigurations/ExampleServer1.xml: -------------------------------------------------------------------------------- 1 | 14 | 15 | 16 | 30 | -------------------------------------------------------------------------------- /dev-support/intellij/runConfigurations/ExampleServer2.xml: -------------------------------------------------------------------------------- 1 | 14 | 15 | 16 | 30 | -------------------------------------------------------------------------------- /dev-support/intellij/runConfigurations/ExampleServer3.xml: -------------------------------------------------------------------------------- 1 | 14 | 15 | 16 | 30 | -------------------------------------------------------------------------------- /dev-support/vagrant/.gitignore: -------------------------------------------------------------------------------- 1 | .*.sw* 2 | *~ 3 | *.box 4 | *.log 5 | .vagrant 6 | -------------------------------------------------------------------------------- /dev-support/vagrant/bin/start_ratis_load_gen.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | # Trivial script to call the Ratis example load gen from GNU Screen 18 | 19 | HOME=/home/vagrant 20 | peers=$1 21 | 22 | cd ${HOME}/ratis 23 | 24 | export QUORUM_OPTS="--peers $peers" 25 | # run the load generator 26 | ./ratis-examples/src/main/bin/client.sh filestore loadgen --size 1048576 --numFiles 100 2>&1 | \ 27 | tee ${HOME}/loadgen.log 28 | 29 | # verify all logs checksum the same 30 | echo "Verification of all Ratis file server logs have the same checksum across all storage directories:" 31 | find ${HOME}/test_data/data? -type f -a -name 'file-*' -exec md5sum \{\} \+ | sed 's/ .*//' | sort | uniq -c 32 | -------------------------------------------------------------------------------- /dev-support/vagrant/bin/start_ratis_server.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 | # Trivial script to call the Ratis example server from GNU Screen 19 | 20 | HOME=/home/vagrant 21 | storage=$1 22 | id=$2 23 | peers=$3 24 | 25 | cd ${HOME}/ratis/ 26 | ./ratis-examples/src/main/bin/server.sh filestore server --storage $storage --id $id --peers $peers 2>&1 | \ 27 | tee ${HOME}/server_${id}.log 28 | -------------------------------------------------------------------------------- /dev-support/vagrant/namazu_configs/hdd_config.toml: -------------------------------------------------------------------------------- 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 | # Policy for a "flakey" disk to stress performance and recovery 17 | explorePolicy = "random" 18 | 19 | [explorePolicyParam] 20 | # for the Filesystem inspector, event will be non-deterministically delayed. 21 | # minInterval and maxInterval are bounds for the non-deterministic delays 22 | minInterval = "10ms" 23 | maxInterval = "300ms" 24 | 25 | # for Filesystem inspectors, you can specify fault-injection probability (0.0-1.0). 26 | faultActionProbability = 0.0 27 | -------------------------------------------------------------------------------- /ratis-assembly/src/main/resources/README.md: -------------------------------------------------------------------------------- 1 | 14 | 15 | # Apache Ratis 16 | 17 | This is the binary distribution of Apache Ratis. 18 | 19 | Apache Ratis is a java library that implements the RAFT protocol. 20 | 21 | First of all: it's a library. To use it use java dependency management tool (such as maven and gradle). 22 | The required artifacts are available from the maven central or the apache nexus under the org.apache.ratis groupId. 23 | 24 | This distribution also includes some example raft server in the examples directory which includes an example 25 | implementation of the key raft elements: raftlog and state machine. The are for demonstration purposes only. 26 | 27 | 28 | For more deails see: 29 | 30 | https://ratis.apache.org 31 | -------------------------------------------------------------------------------- /ratis-client/src/main/java/org/apache/ratis/client/DataStreamOutputRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.client; 19 | 20 | import org.apache.ratis.client.api.DataStreamOutput; 21 | import org.apache.ratis.protocol.DataStreamReply; 22 | 23 | import java.util.concurrent.CompletableFuture; 24 | 25 | /** An RPC interface which extends the user interface {@link DataStreamOutput}. */ 26 | public interface DataStreamOutputRpc extends DataStreamOutput { 27 | /** Get the future of the header request. */ 28 | CompletableFuture getHeaderFuture(); 29 | } 30 | -------------------------------------------------------------------------------- /ratis-client/src/main/java/org/apache/ratis/client/DataStreamRpcApi.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.client; 20 | 21 | import org.apache.ratis.client.api.DataStreamApi; 22 | import org.apache.ratis.client.api.DataStreamOutput; 23 | import org.apache.ratis.protocol.RaftClientRequest; 24 | 25 | /** An RPC interface which extends the user interface {@link DataStreamApi}. */ 26 | public interface DataStreamRpcApi extends DataStreamApi { 27 | /** Create a stream for primary server to send data to peer server. */ 28 | DataStreamOutput stream(RaftClientRequest request); 29 | } 30 | -------------------------------------------------------------------------------- /ratis-client/src/main/java/org/apache/ratis/client/api/LeaderElectionManagementApi.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.client.api; 19 | 20 | import org.apache.ratis.protocol.RaftClientReply; 21 | 22 | import java.io.IOException; 23 | 24 | /** 25 | * An API to support control leader election 26 | * such as pause and resume election 27 | */ 28 | public interface LeaderElectionManagementApi { 29 | 30 | /** pause leader election. */ 31 | RaftClientReply pause() throws IOException; 32 | 33 | /** resume leader election. */ 34 | RaftClientReply resume() throws IOException; 35 | 36 | } 37 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/datastream/DataStreamFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.datastream; 19 | 20 | public interface DataStreamFactory extends DataStreamType.Get { 21 | } 22 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/io/CorruptedFileException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.io; 19 | 20 | import java.io.File; 21 | import java.io.IOException; 22 | 23 | public class CorruptedFileException extends IOException { 24 | public CorruptedFileException(File file, String message) { 25 | super("File " + file + " (exist? " + file.exists() + ", length=" + file.length() + ") is corrupted: " + message); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/io/StandardWriteOption.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.io; 19 | 20 | public enum StandardWriteOption implements WriteOption { 21 | /** 22 | * Sync the data to the underlying storage. 23 | * Note that SYNC does not imply {@link #FLUSH}. 24 | */ 25 | SYNC, 26 | /** Close the data to the underlying storage. */ 27 | CLOSE, 28 | /** Flush the data out from the buffer. */ 29 | FLUSH, 30 | } 31 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamPacket.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.protocol; 20 | 21 | import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type; 22 | 23 | public interface DataStreamPacket { 24 | ClientId getClientId(); 25 | 26 | Type getType(); 27 | 28 | long getStreamId(); 29 | 30 | long getStreamOffset(); 31 | 32 | long getDataLength(); 33 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamReply.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.protocol; 20 | 21 | import org.apache.ratis.proto.RaftProtos.CommitInfoProto; 22 | 23 | import java.util.Collection; 24 | 25 | public interface DataStreamReply extends DataStreamPacket { 26 | 27 | boolean isSuccess(); 28 | 29 | long getBytesWritten(); 30 | 31 | /** @return the commit information when the reply is created. */ 32 | Collection getCommitInfos(); 33 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.protocol; 20 | 21 | import org.apache.ratis.io.WriteOption; 22 | import java.util.List; 23 | 24 | public interface DataStreamRequest extends DataStreamPacket { 25 | List getWriteOptionList(); 26 | 27 | /** @deprecated use {@link #getWriteOptionList()}. */ 28 | @Deprecated 29 | default WriteOption[] getWriteOptions() { 30 | return getWriteOptionList().toArray(WriteOption.EMPTY_ARRAY); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/GroupInfoRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | /** 21 | * Client sends this request to a server to request for the information about 22 | * the server itself. 23 | */ 24 | public class GroupInfoRequest extends RaftClientRequest { 25 | public GroupInfoRequest(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId) { 26 | super(clientId, serverId, groupId, callId, readRequestType()); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/GroupListRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | /** 21 | * Client sends this request to a server to request for the information about 22 | * the server itself. 23 | */ 24 | public class GroupListRequest extends RaftClientRequest { 25 | public GroupListRequest(ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId) { 26 | super(clientId, serverId, groupId, callId, readRequestType()); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientAsynchronousProtocol.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | import java.io.IOException; 21 | import java.util.concurrent.CompletableFuture; 22 | 23 | /** Asynchronous version of {@link RaftClientProtocol}. */ 24 | public interface RaftClientAsynchronousProtocol { 25 | CompletableFuture submitClientRequestAsync( 26 | RaftClientRequest request) throws IOException; 27 | 28 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/RaftClientProtocol.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | import java.io.IOException; 21 | 22 | public interface RaftClientProtocol { 23 | RaftClientReply submitClientRequest(RaftClientRequest request) throws IOException; 24 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/RaftRpcMessage.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | public interface RaftRpcMessage { 21 | 22 | boolean isRequest(); 23 | 24 | String getRequestorId(); 25 | 26 | String getReplierId(); 27 | 28 | RaftGroupId getRaftGroupId(); 29 | } 30 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/TransferLeadershipRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol; 19 | 20 | public class TransferLeadershipRequest extends RaftClientRequest { 21 | private final RaftPeerId newLeader; 22 | 23 | public TransferLeadershipRequest( 24 | ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, RaftPeerId newLeader, long timeoutMs) { 25 | super(clientId, serverId, groupId, callId, readRequestType(), timeoutMs); 26 | this.newLeader = newLeader; 27 | } 28 | 29 | public RaftPeerId getNewLeader() { 30 | return newLeader; 31 | } 32 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/AlreadyClosedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * The corresponding object is already closed. 22 | */ 23 | public class AlreadyClosedException extends RaftException { 24 | public AlreadyClosedException(String message) { 25 | super(message); 26 | } 27 | 28 | public AlreadyClosedException(String message, Throwable t) { 29 | super(message, t); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/AlreadyExistsException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | import java.io.IOException; 21 | 22 | /** 23 | * Signals that an attempt to create a file at a given pathname has failed 24 | * because another file already existed at that path. 25 | */ 26 | public class AlreadyExistsException extends IOException { 27 | private static final long serialVersionUID = 1L; 28 | 29 | public AlreadyExistsException(String msg) { 30 | super(msg); 31 | } 32 | 33 | public AlreadyExistsException(Throwable cause) { 34 | super(cause); 35 | } 36 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ChecksumException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.protocol.exceptions; 20 | 21 | import java.io.IOException; 22 | 23 | /** Thrown for checksum errors. */ 24 | public class ChecksumException extends IOException { 25 | private static final long serialVersionUID = 1L; 26 | private long pos; 27 | public ChecksumException(String description, long pos) { 28 | super(description); 29 | this.pos = pos; 30 | } 31 | 32 | public long getPos() { 33 | return pos; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/DataStreamException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | import org.apache.ratis.protocol.RaftPeerId; 21 | 22 | public class DataStreamException extends RaftException { 23 | public DataStreamException(RaftPeerId peerId, Throwable cause) { 24 | super(cause.getClass().getName() + " from Server " + peerId + ": " + cause.getMessage(), cause); 25 | } 26 | 27 | public DataStreamException(String msg) { 28 | super(msg); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/GroupMismatchException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * This exception indicates that the group id in the request does not match 22 | * server's group id. 23 | */ 24 | public class GroupMismatchException extends RaftException { 25 | public GroupMismatchException(String message) { 26 | super(message); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/LeaderSteppingDownException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | public class LeaderSteppingDownException extends RaftException { 21 | 22 | public LeaderSteppingDownException(String message) { 23 | super(message); 24 | } 25 | 26 | public LeaderSteppingDownException(String message, Throwable t) { 27 | super(message, t); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/RaftException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | import java.io.IOException; 21 | 22 | public class RaftException extends IOException { 23 | private static final long serialVersionUID = 1L; 24 | 25 | public RaftException(String message) { 26 | super(message); 27 | } 28 | 29 | public RaftException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RaftException(String message, Throwable cause) { 34 | super(message, cause); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ReadException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * This exception indicates the failure of a read request. 22 | */ 23 | public class ReadException extends RaftException { 24 | public ReadException(String message) { 25 | super(message); 26 | } 27 | 28 | public ReadException(Throwable cause) { 29 | super(cause); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ReadIndexException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | 21 | /** 22 | * This indicates a retryable read exception 23 | */ 24 | public class ReadIndexException extends RaftException { 25 | 26 | public ReadIndexException(String message) { 27 | super(message); 28 | } 29 | public ReadIndexException(String message, Throwable throwable) { 30 | super(message, throwable); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ReconfigurationInProgressException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | public class ReconfigurationInProgressException extends RaftException { 21 | public ReconfigurationInProgressException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ReconfigurationTimeoutException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | public class ReconfigurationTimeoutException extends RaftException { 21 | public ReconfigurationTimeoutException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ResourceUnavailableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * A particular resource is unavailable. 22 | */ 23 | public class ResourceUnavailableException extends RaftException { 24 | public ResourceUnavailableException(String message) { 25 | super(message); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/ServerNotReadyException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * The server is not ready yet. 22 | */ 23 | public class ServerNotReadyException extends RaftException { 24 | public ServerNotReadyException(String message) { 25 | super(message); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/SetConfigurationException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | public class SetConfigurationException extends RaftException { 21 | 22 | public SetConfigurationException(String message) { 23 | super(message); 24 | } 25 | 26 | public SetConfigurationException(String message, Throwable t) { 27 | super(message, t); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/StaleReadException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * This exception indicates the failure of a stale-read. 22 | */ 23 | public class StaleReadException extends RaftException { 24 | public StaleReadException(String message) { 25 | super(message); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/StreamException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | /** 21 | * A stream is not found in the server. 22 | */ 23 | public class StreamException extends RaftException { 24 | public StreamException(String message) { 25 | super(message); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/TimeoutIOException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | import java.io.IOException; 21 | 22 | /** 23 | * Timeout has occurred for a blocking I/O. 24 | */ 25 | public class TimeoutIOException extends IOException { 26 | static final long serialVersionUID = 1L; 27 | 28 | public TimeoutIOException(String message) { 29 | super(message); 30 | } 31 | 32 | public TimeoutIOException(String message, Throwable throwable) { 33 | super(message, throwable); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/protocol/exceptions/TransferLeadershipException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.protocol.exceptions; 19 | 20 | public class TransferLeadershipException extends RaftException { 21 | 22 | public TransferLeadershipException(String message) { 23 | super(message); 24 | } 25 | 26 | public TransferLeadershipException(String message, Throwable t) { 27 | super(message, t); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/rpc/RpcFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.rpc; 19 | 20 | /** The type of RPC Factory. */ 21 | public interface RpcFactory extends RpcType.Get { 22 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/UncheckedAutoCloseable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util; 19 | 20 | /** 21 | * The same as {@link AutoCloseable} 22 | * except that the close method does not throw {@link Exception}. 23 | */ 24 | @FunctionalInterface 25 | public interface UncheckedAutoCloseable extends AutoCloseable { 26 | @Override 27 | void close(); 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedBiConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** BiConsumer with a throws-clause. */ 21 | @FunctionalInterface 22 | public interface CheckedBiConsumer { 23 | /** 24 | * The same as {@link java.util.function.BiConsumer#accept(Object, Object)} 25 | * except that this method is declared with a throws-clause. 26 | */ 27 | void accept(LEFT left, RIGHT right) throws THROWABLE; 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedBiFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** BiFunction with a throws-clause. */ 21 | @FunctionalInterface 22 | public interface CheckedBiFunction { 23 | /** 24 | * The same as {@link java.util.function.BiFunction#apply(Object, Object)} 25 | * except that this method is declared with a throws-clause. 26 | */ 27 | OUTPUT apply(LEFT left, RIGHT right) throws THROWABLE; 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** Function with a throws-clause. */ 21 | @FunctionalInterface 22 | public interface CheckedFunction { 23 | /** 24 | * The same as {@link java.util.function.Function#apply(Object)} 25 | * except that this method is declared with a throws-clause. 26 | */ 27 | OUTPUT apply(INPUT input) throws THROWABLE; 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedFunctionWithTimeout.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | import org.apache.ratis.util.TimeDuration; 21 | 22 | import java.util.concurrent.TimeoutException; 23 | 24 | /** Function with a timeout and a throws-clause. */ 25 | @FunctionalInterface 26 | public interface CheckedFunctionWithTimeout { 27 | /** 28 | * The same as {@link CheckedFunction#apply(Object)} 29 | * except that this method has a timeout parameter and throws {@link TimeoutException}. 30 | */ 31 | OUTPUT apply(INPUT input, TimeDuration timeout) throws TimeoutException, THROWABLE; 32 | } 33 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedRunnable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** Runnable with a throws-clause. */ 21 | @FunctionalInterface 22 | public interface CheckedRunnable { 23 | /** 24 | * The same as {@link Runnable#run()} 25 | * except that this method is declared with a throws-clause. 26 | */ 27 | void run() throws THROWABLE; 28 | 29 | static CheckedSupplier asCheckedSupplier( 30 | CheckedRunnable runnable) { 31 | return () -> { 32 | runnable.run(); 33 | return null; 34 | }; 35 | } 36 | } -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/CheckedSupplier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** Supplier with a throws-clause. */ 21 | @FunctionalInterface 22 | public interface CheckedSupplier { 23 | /** 24 | * The same as {@link java.util.function.Supplier#get()} 25 | * except that this method is declared with a throws-clause. 26 | */ 27 | OUTPUT get() throws THROWABLE; 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/FunctionUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | import java.util.function.Consumer; 21 | import java.util.function.Function; 22 | 23 | public interface FunctionUtils { 24 | /** 25 | * Convert the given consumer to a function with any output type 26 | * such that the returned function always returns null. 27 | */ 28 | static Function consumerAsNullFunction(Consumer consumer) { 29 | return input -> { 30 | consumer.accept(input); 31 | return null; 32 | }; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/TriConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | /** Consumer with three input parameters. */ 21 | @FunctionalInterface 22 | public interface TriConsumer { 23 | /** 24 | * The same as {@link java.util.function.BiConsumer#accept(Object, Object)}} 25 | * except that this method is declared with three parameters. 26 | */ 27 | void accept(T t, U u, V v); 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/main/java/org/apache/ratis/util/function/UncheckedAutoCloseableSupplier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.util.function; 19 | 20 | import org.apache.ratis.util.UncheckedAutoCloseable; 21 | 22 | import java.util.function.Supplier; 23 | 24 | /** 25 | * A {@link Supplier} which is also {@link UncheckedAutoCloseable}. 26 | * 27 | * @param the type of the {@link Supplier}. 28 | */ 29 | public interface UncheckedAutoCloseableSupplier extends UncheckedAutoCloseable, Supplier { 30 | } 31 | -------------------------------------------------------------------------------- /ratis-common/src/test/java/org/apache/ratis/test/tag/FlakyTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.test.tag; 19 | 20 | /** 21 | * Interface to mark JUnit4 test classes or methods that exhibit intermittent 22 | * issues. These are run separately from the normal tests in CI. In case of 23 | * failure they may be repeated a few times. 24 | * Usage: @Category(FlakyTest.class) @Flaky("RATIS-123") 25 | */ 26 | public interface FlakyTest { 27 | // category marker 28 | } 29 | -------------------------------------------------------------------------------- /ratis-common/src/test/resources/META-INF/services/org.junit.platform.launcher.TestExecutionListener: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | org.apache.ratis.JUnit5TestExecutionListener -------------------------------------------------------------------------------- /ratis-common/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed under the Apache License, Version 2.0 (the "License"); 2 | # you may not use this file except in compliance with the License. 3 | # You may obtain a copy of the License at 4 | # 5 | # http://www.apache.org/licenses/LICENSE-2.0 6 | # 7 | # Unless required by applicable law or agreed to in writing, software 8 | # distributed under the License is distributed on an "AS IS" BASIS, 9 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | # See the License for the specific language governing permissions and 11 | # limitations under the License. 12 | # log4j configuration used during build and unit tests 13 | 14 | log4j.rootLogger=info,stdout 15 | log4j.threshold=ALL 16 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 17 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 18 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 19 | -------------------------------------------------------------------------------- /ratis-docs/README.md: -------------------------------------------------------------------------------- 1 | 17 | # Apache Ratis docs 18 | 19 | This subproject contains the inline documentation for Apache Ratis. 20 | 21 | ## View rendered documents 22 | To view the documents locally, you can run: 23 | 24 | ``` 25 | cd ratis-docs 26 | mvn site:run 27 | ``` 28 | 29 | Then visit http://localhost:8080/ to view rendered documents. 30 | 31 | ## Write document 32 | To create new document, please add markdown files into `src/site/markdown` folder, and then create a link in `site.xml`. For example, `site/markdown/cli.md` could be accessed by `cli.html`. 33 | 34 | For more about the usage, please refer to the [documentation of maven-site-plugin](https://maven.apache.org/guides/mini/guide-site.html). 35 | 36 | -------------------------------------------------------------------------------- /ratis-docs/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 15 | 19 | 4.0.0 20 | 21 | ratis 22 | org.apache.ratis 23 | 3.1.3 24 | 25 | 26 | ratis-docs 27 | Apache Ratis Documentation 28 | jar 29 | 30 | 31 | 32 | true 33 | 34 | 35 | 36 | -------------------------------------------------------------------------------- /ratis-docs/src/site/resources/images/ratis.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/ratis/848cd394c4cc6ed4e1568bfa6c92a922dfe3f10e/ratis-docs/src/site/resources/images/ratis.png -------------------------------------------------------------------------------- /ratis-examples/src/main/bin/client.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" > /dev/null && pwd )" 18 | 19 | USAGE="client.sh " 20 | 21 | if [ "$#" -lt 2 ]; then 22 | echo "$USAGE" 23 | exit 1 24 | fi 25 | 26 | source $DIR/common.sh 27 | 28 | # One of the examples, e.g. "filestore" or "arithmetic" 29 | example="$1" 30 | shift 31 | 32 | subcommand="$1" 33 | shift 34 | 35 | java ${LOGGER_OPTS} -jar $ARTIFACT "$example" "$subcommand" "$@" 36 | -------------------------------------------------------------------------------- /ratis-examples/src/main/bin/server.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" > /dev/null && pwd )" 18 | source $DIR/common.sh 19 | 20 | java ${LOGGER_OPTS} -jar $ARTIFACT "$@" 21 | -------------------------------------------------------------------------------- /ratis-examples/src/main/bin/stop-all.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | kill $(jps | grep 'ratis-examples' | grep -v 'grep' | awk '{print $1}') 18 | echo "All Ratis examples have been stopped." 19 | -------------------------------------------------------------------------------- /ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/Evaluable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.examples.arithmetic; 19 | 20 | import java.util.Map; 21 | 22 | public interface Evaluable { 23 | Double evaluate(Map variableMap); 24 | } 25 | -------------------------------------------------------------------------------- /ratis-examples/src/main/java/org/apache/ratis/examples/arithmetic/cli/Arithmetic.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.examples.arithmetic.cli; 20 | 21 | import org.apache.ratis.examples.common.SubCommandBase; 22 | 23 | import java.util.ArrayList; 24 | import java.util.List; 25 | 26 | /** 27 | * This class enumerates all the commands enqueued by Arithmetic state machine. 28 | */ 29 | public final class Arithmetic { 30 | private Arithmetic() { 31 | } 32 | 33 | public static List getSubCommands() { 34 | List commands = new ArrayList<>(); 35 | commands.add(new Server()); 36 | commands.add(new Assign()); 37 | commands.add(new Get()); 38 | return commands; 39 | } 40 | } -------------------------------------------------------------------------------- /ratis-examples/src/main/resources/conf.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | raft.server.address.list=127.0.0.1:10024,127.0.0.1:10124,127.0.0.1:11124 18 | # raft.server.root.storage.path 19 | # raft.server.priority.list=1,0,0 20 | # raft.server.simulated-slowness.list=0,1s,0 -------------------------------------------------------------------------------- /ratis-examples/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | log4j.rootLogger=INFO, stdout 19 | 20 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 21 | log4j.appender.stdout.Target=System.out 22 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 23 | log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n 24 | -------------------------------------------------------------------------------- /ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreAsyncWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.examples.filestore; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | 22 | public class TestFileStoreAsyncWithGrpc 23 | extends FileStoreAsyncBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreStreamingWithGrpcCluster.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.examples.filestore; 19 | 20 | import org.apache.ratis.datastream.MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty; 21 | 22 | public class TestFileStoreStreamingWithGrpcCluster 23 | extends FileStoreStreamingBaseTest 24 | implements MiniRaftClusterWithRpcTypeGrpcAndDataStreamTypeNetty.FactoryGet { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.examples.filestore; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | 22 | public class TestFileStoreWithGrpc 23 | extends FileStoreBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-examples/src/test/java/org/apache/ratis/examples/filestore/TestFileStoreWithNetty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.examples.filestore; 19 | 20 | import org.apache.ratis.netty.MiniRaftClusterWithNetty; 21 | 22 | public class TestFileStoreWithNetty 23 | extends FileStoreBaseTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | 26 | @Override 27 | public void testWatch() { 28 | //NettyClientRpc does not support sendRequestAsyncUnordered 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /ratis-examples/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed under the Apache License, Version 2.0 (the "License"); 2 | # you may not use this file except in compliance with the License. 3 | # You may obtain a copy of the License at 4 | # 5 | # http://www.apache.org/licenses/LICENSE-2.0 6 | # 7 | # Unless required by applicable law or agreed to in writing, software 8 | # distributed under the License is distributed on an "AS IS" BASIS, 9 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | # See the License for the specific language governing permissions and 11 | # limitations under the License. 12 | # log4j configuration used during build and unit tests 13 | 14 | log4j.rootLogger=info,stdout 15 | log4j.threshold=ALL 16 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 17 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 18 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n 19 | -------------------------------------------------------------------------------- /ratis-experiments/scripts/flatbuf-compile.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://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 | set -eu 18 | 19 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" > /dev/null && pwd )" 20 | DOCDIR="$DIR/.." 21 | 22 | if [ "$(which flatc)" ]; then 23 | DESTDIR="$DOCDIR/target/generated-sources/" 24 | mkdir -p "$DESTDIR" 25 | cd "$DOCDIR" 26 | flatc --grpc --java --gen-mutable -o ./target/generated-sources/ ./src/main/flatbufs/*.fbs 27 | cd - 28 | fi 29 | 30 | 31 | -------------------------------------------------------------------------------- /ratis-experiments/src/main/flatbufs/FileTransfer.fbs: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | namespace org.apache.ratis.flatbufs; 19 | 20 | table TransferMsg { 21 | partId: uint32; 22 | data: [byte]; 23 | } 24 | 25 | table TransferReply { 26 | partId: uint32; 27 | message: string; 28 | } 29 | 30 | rpc_service FileTransfer{ 31 | SendData(TransferMsg):TransferReply (streaming: "bidi"); 32 | } 33 | 34 | root_type TransferMsg; 35 | -------------------------------------------------------------------------------- /ratis-experiments/src/main/java/org/apache/ratis/experiments/nettyzerocopy/objects/RequestDataComposite.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.experiments.nettyzerocopy.objects; 20 | 21 | import org.apache.ratis.thirdparty.io.netty.buffer.ByteBuf; 22 | 23 | public class RequestDataComposite { 24 | private int dataId; 25 | private ByteBuf buff; 26 | 27 | public ByteBuf getBuff() { 28 | return buff; 29 | } 30 | 31 | public int getDataId() { 32 | return dataId; 33 | } 34 | 35 | public void setDataId(int dataId) { 36 | this.dataId = dataId; 37 | } 38 | 39 | public void setBuff(ByteBuf buff) { 40 | this.buff = buff; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /ratis-experiments/src/main/java/org/apache/ratis/experiments/nettyzerocopy/objects/ResponseData.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.experiments.nettyzerocopy.objects; 20 | 21 | public class ResponseData { 22 | private int id; 23 | 24 | public int getId() { 25 | return id; 26 | } 27 | 28 | public void setId(int id) { 29 | this.id = id; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ratis-metrics-api/src/main/java/org/apache/ratis/metrics/LongCounter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.metrics; 19 | 20 | public interface LongCounter { 21 | default void inc() { 22 | inc(1L); 23 | } 24 | 25 | void inc(long n); 26 | 27 | default void dec() { 28 | dec(1L); 29 | } 30 | 31 | void dec(long n); 32 | 33 | long getCount(); 34 | } 35 | -------------------------------------------------------------------------------- /ratis-metrics-api/src/main/java/org/apache/ratis/metrics/MetricRegistryFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.metrics; 19 | 20 | /** 21 | * A Factory for creating MetricRegistries. This is the main plugin point for metrics implementation 22 | */ 23 | public interface MetricRegistryFactory { 24 | /** 25 | * Create a MetricRegistry from the given MetricRegistryInfo 26 | * @param info the descriptor for MetricRegistry 27 | * @return a MetricRegistry implementation 28 | */ 29 | RatisMetricRegistry create(MetricRegistryInfo info); 30 | } 31 | -------------------------------------------------------------------------------- /ratis-metrics-api/src/main/java/org/apache/ratis/metrics/RatisMetricRegistry.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.metrics; 19 | 20 | import java.util.function.Supplier; 21 | 22 | public interface RatisMetricRegistry { 23 | Timekeeper timer(String name); 24 | 25 | LongCounter counter(String name); 26 | 27 | boolean remove(String name); 28 | 29 | void gauge(String name, Supplier> gaugeSupplier); 30 | 31 | MetricRegistryInfo getMetricRegistryInfo(); 32 | } 33 | -------------------------------------------------------------------------------- /ratis-metrics-default/src/main/java/org/apache/ratis/metrics/impl/MetricRegistryFactoryImpl.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.metrics.impl; 19 | 20 | import org.apache.ratis.metrics.MetricRegistryFactory; 21 | import org.apache.ratis.metrics.MetricRegistryInfo; 22 | 23 | public class MetricRegistryFactoryImpl implements MetricRegistryFactory { 24 | @Override 25 | public RatisMetricRegistryImpl create(MetricRegistryInfo info) { 26 | return new RatisMetricRegistryImpl(info); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-metrics-default/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with 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 | org.apache.ratis.metrics.impl.MetricRegistriesImpl 19 | -------------------------------------------------------------------------------- /ratis-metrics-dropwizard3/src/main/java/org/apache/ratis/metrics/dropwizard3/Dm3MetricRegistryFactoryImpl.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.metrics.dropwizard3; 19 | 20 | import org.apache.ratis.metrics.MetricRegistryFactory; 21 | import org.apache.ratis.metrics.MetricRegistryInfo; 22 | 23 | public class Dm3MetricRegistryFactoryImpl implements MetricRegistryFactory { 24 | @Override 25 | public Dm3RatisMetricRegistryImpl create(MetricRegistryInfo info) { 26 | return new Dm3RatisMetricRegistryImpl(info); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-metrics-dropwizard3/src/main/resources/META-INF/services/org.apache.ratis.metrics.MetricRegistries: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with 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 | org.apache.ratis.metrics.dropwizard3.Dm3MetricRegistriesImpl 19 | -------------------------------------------------------------------------------- /ratis-proto/src/main/proto/Experiments.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | syntax = "proto3"; 19 | option java_package = "org.apache.ratis.proto"; 20 | option java_outer_classname = "ExperimentsProtos"; 21 | option java_generate_equals_and_hash = true; 22 | package ratis.experiments; 23 | 24 | message TransferMsgProto { 25 | uint32 partId = 1; 26 | bytes data = 2; 27 | } 28 | 29 | message TransferReplyProto { 30 | uint32 partId = 1; 31 | string message = 2; 32 | } 33 | 34 | service FileTransferExampleService { 35 | rpc sendData(stream TransferMsgProto) returns (stream TransferReplyProto); 36 | } -------------------------------------------------------------------------------- /ratis-proto/src/main/proto/Test.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | syntax = "proto3"; 19 | 20 | option java_multiple_files = true; 21 | option java_package = "org.apache.ratis.test.proto"; 22 | option java_outer_classname = "TestProto"; 23 | 24 | package org.apache.ratis.test; 25 | 26 | service Greeter { 27 | rpc Hello (stream HelloRequest) 28 | returns (stream HelloReply) {} 29 | 30 | rpc Binary (stream BinaryRequest) 31 | returns (stream BinaryReply) {} 32 | } 33 | 34 | message HelloRequest { 35 | string name = 1; 36 | } 37 | 38 | message HelloReply { 39 | string message = 1; 40 | } 41 | 42 | message BinaryRequest { 43 | bytes data = 1; 44 | } 45 | 46 | message BinaryReply { 47 | bytes data = 1; 48 | } 49 | 50 | -------------------------------------------------------------------------------- /ratis-replicated-map/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed under the Apache License, Version 2.0 (the "License"); 2 | # you may not use this file except in compliance with the License. 3 | # You may obtain a copy of the License at 4 | # 5 | # http://www.apache.org/licenses/LICENSE-2.0 6 | # 7 | # Unless required by applicable law or agreed to in writing, software 8 | # distributed under the License is distributed on an "AS IS" BASIS, 9 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | # See the License for the specific language governing permissions and 11 | # limitations under the License. 12 | # log4j configuration used during build and unit tests 13 | 14 | log4j.rootLogger=info,stdout 15 | log4j.threshold=ALL 16 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 17 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 18 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n 19 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server; 19 | 20 | import java.io.Closeable; 21 | 22 | /** 23 | * Interface for streaming server. 24 | */ 25 | public interface DataStreamServer extends Closeable { 26 | /** 27 | * Get network interface for server. 28 | */ 29 | DataStreamServerRpc getServerRpc(); 30 | } 31 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/DataStreamServerRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server; 19 | 20 | import org.apache.ratis.protocol.RaftPeer; 21 | 22 | import java.io.Closeable; 23 | 24 | /** 25 | * A server interface handling incoming streams 26 | * Relays those streams to other servers after persisting 27 | */ 28 | public interface DataStreamServerRpc extends ServerRpc, RaftPeer.Add, Closeable { 29 | } 30 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/ServerRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server; 19 | 20 | import java.io.Closeable; 21 | import java.io.IOException; 22 | import java.net.InetSocketAddress; 23 | 24 | /** 25 | * A general server interface. 26 | */ 27 | public interface ServerRpc extends Closeable { 28 | /** Start the RPC service. */ 29 | void start() throws IOException; 30 | 31 | /** @return the address where this RPC server is listening to. */ 32 | InetSocketAddress getInetSocketAddress(); 33 | } 34 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/metrics/RaftLogMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.metrics; 19 | 20 | import org.apache.ratis.server.raftlog.LogEntryHeader; 21 | 22 | /** Metrics for a raft log. */ 23 | public interface RaftLogMetrics { 24 | /** A log entry just has been committed. */ 25 | void onLogEntryCommitted(LogEntryHeader header); 26 | 27 | /** Read statemachine data timeout */ 28 | default void onStateMachineDataReadTimeout() { 29 | } 30 | 31 | /** Write statemachine data timeout */ 32 | default void onStateMachineDataWriteTimeout() { 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/metrics/RaftServerMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.server.metrics; 20 | 21 | /** Metrics for a raft Server. */ 22 | public interface RaftServerMetrics { 23 | /** A snapshot just has been installed. */ 24 | void onSnapshotInstalled(); 25 | } -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/raftlog/RaftLogIOException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.raftlog; 19 | 20 | import org.apache.ratis.protocol.exceptions.RaftException; 21 | 22 | /** 23 | * Exception while reading/writing RaftLog 24 | */ 25 | public class RaftLogIOException extends RaftException { 26 | public RaftLogIOException(Throwable cause) { 27 | super(cause); 28 | } 29 | 30 | public RaftLogIOException(String msg) { 31 | super(msg); 32 | } 33 | 34 | public RaftLogIOException(String message, Throwable cause) { 35 | super(message, cause); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/server/storage/RaftStorageMetadataFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.storage; 19 | 20 | import java.io.IOException; 21 | 22 | /** 23 | * Represent a file on disk which persistently stores the metadata of a raft storage. 24 | * The file is updated atomically. 25 | */ 26 | public interface RaftStorageMetadataFile { 27 | /** @return the metadata persisted in this file. */ 28 | RaftStorageMetadata getMetadata() throws IOException; 29 | 30 | /** Persist the given metadata. */ 31 | void persist(RaftStorageMetadata newMetadata) throws IOException; 32 | } 33 | -------------------------------------------------------------------------------- /ratis-server-api/src/main/java/org/apache/ratis/statemachine/SnapshotRetentionPolicy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.statemachine; 19 | 20 | /** 21 | * Retention policy of state machine snapshots. 22 | */ 23 | public interface SnapshotRetentionPolicy { 24 | int DEFAULT_ALL_SNAPSHOTS_RETAINED = -1; 25 | 26 | /** 27 | * @return -1 for retaining all the snapshots; otherwise, return the number of snapshots to be retained. 28 | */ 29 | default int getNumSnapshotsRetained() { 30 | return DEFAULT_ALL_SNAPSHOTS_RETAINED; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/datastream/TestNettyDataStreamWithNettyCluster.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.datastream; 19 | 20 | 21 | import org.junit.jupiter.api.Disabled; 22 | 23 | @Disabled("Ignored by runzhiwang, because NettyClientRpc does not support sendRequestAsync") 24 | public class TestNettyDataStreamWithNettyCluster 25 | extends DataStreamClusterTests 26 | implements MiniRaftClusterWithRpcTypeNettyAndDataStreamTypeNetty.FactoryGet { 27 | } 28 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupInfoWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.GroupInfoBaseTest; 21 | 22 | public class TestGroupInfoWithGrpc 23 | extends GroupInfoBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestGroupManagementWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.MiniRaftCluster; 21 | import org.apache.ratis.server.impl.GroupManagementBaseTest; 22 | 23 | public class TestGroupManagementWithGrpc extends GroupManagementBaseTest { 24 | @Override 25 | public MiniRaftCluster.Factory getClusterFactory() { 26 | return MiniRaftClusterWithGrpc.FACTORY; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestGrpcFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.BaseTest; 21 | import org.junit.jupiter.api.Assertions; 22 | import org.junit.jupiter.api.Test; 23 | 24 | public class TestGrpcFactory extends BaseTest { 25 | @Test 26 | public void testUseCacheForAllThreads() { 27 | // trigger GrpcFactory static initializer 28 | final boolean value = GrpcFactory.checkPooledByteBufAllocatorUseCacheForAllThreads(LOG::info); 29 | Assertions.assertFalse(value); 30 | } 31 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestInstallSnapshotNotificationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.InstallSnapshotNotificationTests; 21 | 22 | public class TestInstallSnapshotNotificationWithGrpc 23 | extends InstallSnapshotNotificationTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestLogMetadataTestsWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.ratis.grpc; 21 | 22 | import org.apache.ratis.server.impl.LogMetadataTests; 23 | 24 | public class TestLogMetadataTestsWithGrpc extends LogMetadataTests 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | 27 | } 28 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestMessageStreamApiWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.MessageStreamApiTests; 21 | 22 | public class TestMessageStreamApiWithGrpc extends MessageStreamApiTests 23 | implements MiniRaftClusterWithGrpc.FactoryGet { 24 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestPreAppendLeaderStepDownWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.PreAppendLeaderStepDownTest; 21 | 22 | public class TestPreAppendLeaderStepDownWithGrpc 23 | extends PreAppendLeaderStepDownTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncExceptionWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.RaftAsyncExceptionTests; 21 | 22 | public class TestRaftAsyncExceptionWithGrpc 23 | extends RaftAsyncExceptionTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftAsyncWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.RaftAsyncTests; 21 | import org.junit.jupiter.api.Timeout; 22 | 23 | @Timeout(100) 24 | public class TestRaftAsyncWithGrpc extends RaftAsyncTests 25 | implements MiniRaftClusterWithGrpc.FactoryGet { 26 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftExceptionWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.RaftExceptionBaseTest; 21 | 22 | public class TestRaftExceptionWithGrpc 23 | extends RaftExceptionBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftLogTruncateWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.RaftLogTruncateTests; 21 | 22 | public class TestRaftLogTruncateWithGrpc extends RaftLogTruncateTests 23 | implements MiniRaftClusterWithGrpc.FactoryGet { 24 | } 25 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftOutputStreamWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.OutputStreamBaseTest; 21 | import org.junit.jupiter.api.Timeout; 22 | 23 | @Timeout(value = 100) 24 | public class TestRaftOutputStreamWithGrpc 25 | extends OutputStreamBaseTest 26 | implements MiniRaftClusterWithGrpc.FactoryGet { 27 | } 28 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftReconfigurationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.RaftReconfigurationBaseTest; 21 | 22 | public class TestRaftReconfigurationWithGrpc 23 | extends RaftReconfigurationBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRaftStateMachineExceptionWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.RaftStateMachineExceptionTests; 21 | 22 | public class TestRaftStateMachineExceptionWithGrpc 23 | extends RaftStateMachineExceptionTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestReConfigPropertyWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | 19 | package org.apache.ratis.grpc; 20 | 21 | 22 | import org.apache.ratis.TestReConfigProperty; 23 | 24 | public class TestReConfigPropertyWithGrpc extends TestReConfigProperty 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestReadOnlyRequestWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.ReadOnlyRequestTests; 21 | 22 | public class TestReadOnlyRequestWithGrpc 23 | extends ReadOnlyRequestTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestReadOnlyRequestWithLongTimeoutWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.ReadOnlyRequestWithLongTimeoutTests; 21 | 22 | public class TestReadOnlyRequestWithLongTimeoutWithGrpc 23 | extends ReadOnlyRequestWithLongTimeoutTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestRequestLimitAsyncWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.RequestLimitAsyncBaseTest; 21 | 22 | public class TestRequestLimitAsyncWithGrpc 23 | extends RequestLimitAsyncBaseTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestServerPauseResumeWithGrpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.ServerPauseResumeTest; 21 | 22 | public class TestServerPauseResumeWithGrpc 23 | extends ServerPauseResumeTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | 26 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestServerRestartWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.ServerRestartTests; 21 | 22 | public class TestServerRestartWithGrpc 23 | extends ServerRestartTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestSnapshotManagementWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.statemachine.SnapshotManagementTest; 21 | 22 | public class TestSnapshotManagementWithGrpc 23 | extends SnapshotManagementTest 24 | implements MiniRaftClusterWithGrpc.FactoryGet { 25 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/TestStateMachineShutdownWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc; 19 | 20 | import org.apache.ratis.server.impl.StateMachineShutdownTests; 21 | 22 | public class TestStateMachineShutdownWithGrpc 23 | extends StateMachineShutdownTests 24 | implements MiniRaftClusterWithGrpc.FactoryGet{ 25 | 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/cli/TestElectionCommandIntegrationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc.cli; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | import org.apache.ratis.shell.cli.sh.ElectionCommandIntegrationTest; 22 | 23 | public class TestElectionCommandIntegrationWithGrpc 24 | extends ElectionCommandIntegrationTest 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/cli/TestGroupCommandIntegrationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc.cli; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | import org.apache.ratis.shell.cli.sh.GroupCommandIntegrationTest; 22 | 23 | public class TestGroupCommandIntegrationWithGrpc 24 | extends GroupCommandIntegrationTest 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/cli/TestPeerCommandIntegrationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc.cli; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | import org.apache.ratis.shell.cli.sh.PeerCommandIntegrationTest; 22 | 23 | public class TestPeerCommandIntegrationWithGrpc 24 | extends PeerCommandIntegrationTest 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/grpc/cli/TestSnapshotCommandIntegrationWithGrpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.grpc.cli; 19 | 20 | import org.apache.ratis.grpc.MiniRaftClusterWithGrpc; 21 | import org.apache.ratis.shell.cli.sh.SnapshotCommandIntegrationTest; 22 | 23 | public class TestSnapshotCommandIntegrationWithGrpc 24 | extends SnapshotCommandIntegrationTest 25 | implements MiniRaftClusterWithGrpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestGroupInfoWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.GroupInfoBaseTest; 21 | 22 | public class TestGroupInfoWithNetty 23 | extends GroupInfoBaseTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestGroupManagementWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.MiniRaftCluster; 21 | import org.apache.ratis.server.impl.GroupManagementBaseTest; 22 | 23 | public class TestGroupManagementWithNetty extends GroupManagementBaseTest { 24 | @Override 25 | public MiniRaftCluster.Factory getClusterFactory() { 26 | return MiniRaftClusterWithNetty.FACTORY; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestLogAppenderWithNetty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.LogAppenderTests; 21 | 22 | public class TestLogAppenderWithNetty 23 | extends LogAppenderTests 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestPreAppendLeaderStepDownWithNetty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.PreAppendLeaderStepDownTest; 21 | 22 | public class TestPreAppendLeaderStepDownWithNetty 23 | extends PreAppendLeaderStepDownTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftAsyncWithNetty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.RaftAsyncTests; 21 | import org.junit.jupiter.api.Timeout; 22 | 23 | @Timeout(100) 24 | public class TestRaftAsyncWithNetty 25 | extends RaftAsyncTests 26 | implements MiniRaftClusterWithNetty.FactoryGet { 27 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftExceptionWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.RaftExceptionBaseTest; 21 | 22 | public class TestRaftExceptionWithNetty 23 | extends RaftExceptionBaseTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftReconfigurationWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.RaftReconfigurationBaseTest; 21 | 22 | public class TestRaftReconfigurationWithNetty 23 | extends RaftReconfigurationBaseTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftSnapshotWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.MiniRaftCluster; 21 | import org.apache.ratis.statemachine.RaftSnapshotBaseTest; 22 | 23 | public class TestRaftSnapshotWithNetty extends RaftSnapshotBaseTest { 24 | @Override 25 | public MiniRaftCluster.Factory getFactory() { 26 | return MiniRaftClusterWithNetty.FACTORY; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftStateMachineExceptionWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.RaftStateMachineExceptionTests; 21 | 22 | public class TestRaftStateMachineExceptionWithNetty 23 | extends RaftStateMachineExceptionTests 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRaftWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.RaftBasicTests; 21 | import org.apache.ratis.server.impl.BlockRequestHandlingInjection; 22 | import org.junit.jupiter.api.Test; 23 | 24 | public class TestRaftWithNetty 25 | extends RaftBasicTests 26 | implements MiniRaftClusterWithNetty.FactoryGet { 27 | 28 | @Override 29 | @Test 30 | public void testWithLoad() throws Exception { 31 | super.testWithLoad(); 32 | BlockRequestHandlingInjection.getInstance().unblockAll(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestRetryCacheWithNettyRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.RetryCacheTests; 21 | 22 | public class TestRetryCacheWithNettyRpc 23 | extends RetryCacheTests 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestServerPauseResumeWithNetty.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.impl.ServerPauseResumeTest; 21 | 22 | public class TestServerPauseResumeWithNetty 23 | extends ServerPauseResumeTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestServerRestartWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.server.ServerRestartTests; 21 | 22 | public class TestServerRestartWithNetty 23 | extends ServerRestartTests 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/netty/TestSnapshotManagementWithNetty.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.netty; 19 | 20 | import org.apache.ratis.statemachine.SnapshotManagementTest; 21 | 22 | public class TestSnapshotManagementWithNetty 23 | extends SnapshotManagementTest 24 | implements MiniRaftClusterWithNetty.FactoryGet { 25 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/protocol/ProtocolTestUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.protocol; 19 | 20 | import org.apache.ratis.util.BiWeakValueCache; 21 | 22 | public interface ProtocolTestUtils { 23 | static BiWeakValueCache getTermIndexCache() { 24 | return TermIndex.Impl.getCache(); 25 | } 26 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupInfoWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.GroupInfoBaseTest; 21 | 22 | public class TestGroupInfoWithSimulatedRpc 23 | extends GroupInfoBaseTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestGroupManagementWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.MiniRaftCluster; 21 | import org.apache.ratis.server.impl.GroupManagementBaseTest; 22 | 23 | public class TestGroupManagementWithSimulatedRpc extends GroupManagementBaseTest { 24 | @Override 25 | public MiniRaftCluster.Factory getClusterFactory() { 26 | return MiniRaftClusterWithSimulatedRpc.FACTORY; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLeaderElectionWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.LeaderElectionTests; 21 | 22 | public class TestLeaderElectionWithSimulatedRpc 23 | extends LeaderElectionTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestLogAppenderWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.LogAppenderTests; 21 | 22 | public class TestLogAppenderWithSimulatedRpc 23 | extends LogAppenderTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestPreAppendLeaderStepDownWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.PreAppendLeaderStepDownTest; 21 | 22 | public class TestPreAppendLeaderStepDownWithSimulatedRpc 23 | extends PreAppendLeaderStepDownTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftExceptionWithSimulation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.RaftExceptionBaseTest; 21 | 22 | public class TestRaftExceptionWithSimulation 23 | extends RaftExceptionBaseTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftReconfigurationWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.RaftReconfigurationBaseTest; 21 | 22 | public class TestRaftReconfigurationWithSimulatedRpc 23 | extends RaftReconfigurationBaseTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftSnapshotWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.MiniRaftCluster; 21 | import org.apache.ratis.statemachine.RaftSnapshotBaseTest; 22 | 23 | public class TestRaftSnapshotWithSimulatedRpc extends RaftSnapshotBaseTest { 24 | @Override 25 | public MiniRaftCluster.Factory getFactory() { 26 | return MiniRaftClusterWithSimulatedRpc.FACTORY; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftStateMachineExceptionWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.RaftStateMachineExceptionTests; 21 | 22 | public class TestRaftStateMachineExceptionWithSimulatedRpc extends 23 | RaftStateMachineExceptionTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRaftWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.RaftBasicTests; 21 | 22 | public class TestRaftWithSimulatedRpc 23 | extends RaftBasicTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRatisServerMetricsWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.impl.TestRatisServerMetricsBase; 21 | 22 | /** Tests on Ratis server metrics with simulated rpc. */ 23 | public class TestRatisServerMetricsWithSimulatedRpc 24 | extends TestRatisServerMetricsBase 25 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet{ 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestRetryCacheWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.RetryCacheTests; 21 | 22 | public class TestRetryCacheWithSimulatedRpc 23 | extends RetryCacheTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerPauseResumeWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | package org.apache.ratis.server.simulation; 2 | 3 | import org.apache.ratis.server.impl.ServerPauseResumeTest; 4 | 5 | /* 6 | * Licensed to the Apache Software Foundation (ASF) under one 7 | * or more contributor license agreements. See the NOTICE file 8 | * distributed with this work for additional information 9 | * regarding copyright ownership. The ASF licenses this file 10 | * to you under the Apache License, Version 2.0 (the 11 | * "License"); you may not use this file except in compliance 12 | * with the License. You may obtain a copy of the License at 13 | * 14 | * http://www.apache.org/licenses/LICENSE-2.0 15 | * 16 | * Unless required by applicable law or agreed to in writing, software 17 | * distributed under the License is distributed on an "AS IS" BASIS, 18 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 19 | * See the License for the specific language governing permissions and 20 | * limitations under the License. 21 | */ 22 | public class TestServerPauseResumeWithSimulatedRpc 23 | extends ServerPauseResumeTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestServerRestartWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.server.ServerRestartTests; 21 | 22 | public class TestServerRestartWithSimulatedRpc 23 | extends ServerRestartTests 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } -------------------------------------------------------------------------------- /ratis-test/src/test/java/org/apache/ratis/server/simulation/TestSnapshotManagementWithSimulatedRpc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 | package org.apache.ratis.server.simulation; 19 | 20 | import org.apache.ratis.statemachine.SnapshotManagementTest; 21 | 22 | public class TestSnapshotManagementWithSimulatedRpc 23 | extends SnapshotManagementTest 24 | implements MiniRaftClusterWithSimulatedRpc.FactoryGet { 25 | } 26 | -------------------------------------------------------------------------------- /ratis-test/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Licensed under the Apache License, Version 2.0 (the "License"); 2 | # you may not use this file except in compliance with the License. 3 | # You may obtain a copy of the License at 4 | # 5 | # http://www.apache.org/licenses/LICENSE-2.0 6 | # 7 | # Unless required by applicable law or agreed to in writing, software 8 | # distributed under the License is distributed on an "AS IS" BASIS, 9 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | # See the License for the specific language governing permissions and 11 | # limitations under the License. 12 | # log4j configuration used during build and unit tests 13 | 14 | log4j.rootLogger=info,stdout 15 | log4j.threshold=ALL 16 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 17 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 18 | log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n 19 | -------------------------------------------------------------------------------- /ratis-test/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker: -------------------------------------------------------------------------------- 1 | # Licensed under the Apache License, Version 2.0 (the "License"); 2 | # you may not use this file except in compliance with the License. 3 | # You may obtain a copy of the License at 4 | # 5 | # http://www.apache.org/licenses/LICENSE-2.0 6 | # 7 | # Unless required by applicable law or agreed to in writing, software 8 | # distributed under the License is distributed on an "AS IS" BASIS, 9 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | # See the License for the specific language governing permissions and 11 | # limitations under the License. 12 | 13 | mock-maker-inline 14 | -------------------------------------------------------------------------------- /src/main/resources/ratis-version.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with 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 | name=${project.name} 19 | version=${project.version} 20 | revision=${version-info.scm.commit} 21 | --------------------------------------------------------------------------------