├── .gitignore ├── README.md ├── build.sbt ├── project ├── Dependencies.scala ├── Libs.scala ├── Settings.scala ├── build.properties ├── plugins.sbt ├── project │ └── metaplugins.sbt └── publishing.scala └── src ├── main ├── go │ ├── README.md │ ├── consensus │ ├── go.mod │ ├── go.sum │ ├── kvstore │ │ ├── embed │ │ │ └── Config.go │ │ ├── idutil │ │ │ ├── id.go │ │ │ └── id_test.go │ │ ├── metrics.go │ │ ├── runtime │ │ │ └── fds_linux.go │ │ ├── server.go │ │ ├── server_test.go │ │ ├── storage.go │ │ └── types │ │ │ ├── id.go │ │ │ └── urls.go │ ├── main.go │ ├── mvcc │ │ ├── index.go │ │ ├── index_test.go │ │ ├── key_index.go │ │ ├── key_index_test.go │ │ ├── metrics.go │ │ ├── revision.go │ │ ├── revision_test.go │ │ └── store.go │ ├── mvccpb │ │ ├── kv.pb.go │ │ └── kv.proto │ ├── newraft │ │ ├── README.md │ │ ├── confchange │ │ │ ├── confchange.go │ │ │ ├── datadriven_test.go │ │ │ ├── quick_test.go │ │ │ ├── restore.go │ │ │ ├── restore_test.go │ │ │ └── testdata │ │ │ │ ├── joint_autoleave.txt │ │ │ │ ├── joint_idempotency.txt │ │ │ │ ├── joint_learners_next.txt │ │ │ │ ├── joint_safety.txt │ │ │ │ ├── simple_idempotency.txt │ │ │ │ ├── simple_promote_demote.txt │ │ │ │ ├── simple_safety.txt │ │ │ │ ├── update.txt │ │ │ │ └── zero.txt │ │ ├── crc │ │ │ ├── crc.go │ │ │ └── crc_test.go │ │ ├── example │ │ │ └── raft.go │ │ ├── fileutil │ │ │ ├── dir_unix.go │ │ │ ├── doc.go │ │ │ ├── fileutil.go │ │ │ ├── lock.go │ │ │ ├── lock_flock.go │ │ │ ├── lock_linux.go │ │ │ ├── preallocate.go │ │ │ ├── preallocate_unix.go │ │ │ ├── read_dir.go │ │ │ ├── sync.go │ │ │ └── sync_linux.go │ │ ├── ioutil │ │ │ ├── pagewriter.go │ │ │ ├── pagewriter_test.go │ │ │ ├── readcloser.go │ │ │ ├── readcloser_test.go │ │ │ ├── reader.go │ │ │ ├── reader_test.go │ │ │ └── util.go │ │ ├── log.go │ │ ├── log_unstable.go │ │ ├── logger.go │ │ ├── member.go │ │ ├── node.go │ │ ├── pbutil │ │ │ ├── pbutil.go │ │ │ └── pbutil_test.go │ │ ├── quorum │ │ │ ├── bench_test.go │ │ │ ├── datadriven_test.go │ │ │ ├── joint.go │ │ │ ├── majority.go │ │ │ ├── quick_test.go │ │ │ ├── quorum.go │ │ │ ├── testdata │ │ │ │ ├── joint_commit.txt │ │ │ │ ├── joint_vote.txt │ │ │ │ ├── majority_commit.txt │ │ │ │ └── majority_vote.txt │ │ │ └── voteresult_string.go │ │ ├── raft.go │ │ ├── raftpb │ │ │ ├── confchange.go │ │ │ ├── confstate.go │ │ │ ├── raft.pb.go │ │ │ └── raft.proto │ │ ├── rawnode.go │ │ ├── read_only.go │ │ ├── status.go │ │ ├── storage.go │ │ ├── tracker │ │ │ ├── inflights.go │ │ │ ├── inflights_test.go │ │ │ ├── progress.go │ │ │ ├── progress_test.go │ │ │ ├── state.go │ │ │ └── tracker.go │ │ ├── util.go │ │ └── wal │ │ │ ├── decoder.go │ │ │ ├── encoder.go │ │ │ ├── file_pipeline.go │ │ │ ├── file_pipeline_test.go │ │ │ ├── util.go │ │ │ ├── wal.go │ │ │ ├── wal_test.go │ │ │ └── walpb │ │ │ ├── record.go │ │ │ ├── record.pb.go │ │ │ └── record.proto │ ├── raft │ │ ├── append_entries.go │ │ ├── debug.go │ │ ├── log.go │ │ ├── log_entry.go │ │ ├── log_entry_test.go │ │ ├── peer.go │ │ ├── raftpb │ │ │ ├── append_entries_request.pb.go │ │ │ ├── append_entries_request.proto │ │ │ ├── append_entries_responses.pb.go │ │ │ ├── append_entries_responses.proto │ │ │ ├── log_entry.pb.go │ │ │ ├── log_entry.proto │ │ │ ├── request_vote_request.pb.go │ │ │ ├── request_vote_request.proto │ │ │ ├── request_vote_responses.pb.go │ │ │ └── request_vote_responses.proto │ │ ├── server.go │ │ ├── server_test.go │ │ ├── storage.go │ │ ├── storage_test.go │ │ ├── transporter.go │ │ └── util.go │ └── vendor │ │ ├── github.com │ │ ├── davecgh │ │ │ └── go-spew │ │ │ │ ├── LICENSE │ │ │ │ └── spew │ │ │ │ ├── bypass.go │ │ │ │ ├── bypasssafe.go │ │ │ │ ├── common.go │ │ │ │ ├── config.go │ │ │ │ ├── doc.go │ │ │ │ ├── dump.go │ │ │ │ ├── format.go │ │ │ │ └── spew.go │ │ ├── gogo │ │ │ └── protobuf │ │ │ │ ├── AUTHORS │ │ │ │ ├── CONTRIBUTORS │ │ │ │ ├── LICENSE │ │ │ │ ├── gogoproto │ │ │ │ ├── Makefile │ │ │ │ ├── doc.go │ │ │ │ ├── gogo.pb.go │ │ │ │ ├── gogo.pb.golden │ │ │ │ ├── gogo.proto │ │ │ │ └── helper.go │ │ │ │ ├── proto │ │ │ │ ├── Makefile │ │ │ │ ├── clone.go │ │ │ │ ├── custom_gogo.go │ │ │ │ ├── decode.go │ │ │ │ ├── deprecated.go │ │ │ │ ├── discard.go │ │ │ │ ├── duration.go │ │ │ │ ├── duration_gogo.go │ │ │ │ ├── encode.go │ │ │ │ ├── encode_gogo.go │ │ │ │ ├── equal.go │ │ │ │ ├── extensions.go │ │ │ │ ├── extensions_gogo.go │ │ │ │ ├── lib.go │ │ │ │ ├── lib_gogo.go │ │ │ │ ├── message_set.go │ │ │ │ ├── pointer_reflect.go │ │ │ │ ├── pointer_reflect_gogo.go │ │ │ │ ├── pointer_unsafe.go │ │ │ │ ├── pointer_unsafe_gogo.go │ │ │ │ ├── properties.go │ │ │ │ ├── properties_gogo.go │ │ │ │ ├── skip_gogo.go │ │ │ │ ├── table_marshal.go │ │ │ │ ├── table_marshal_gogo.go │ │ │ │ ├── table_merge.go │ │ │ │ ├── table_unmarshal.go │ │ │ │ ├── table_unmarshal_gogo.go │ │ │ │ ├── text.go │ │ │ │ ├── text_gogo.go │ │ │ │ ├── text_parser.go │ │ │ │ ├── timestamp.go │ │ │ │ ├── timestamp_gogo.go │ │ │ │ ├── wrappers.go │ │ │ │ └── wrappers_gogo.go │ │ │ │ └── protoc-gen-gogo │ │ │ │ └── descriptor │ │ │ │ ├── Makefile │ │ │ │ ├── descriptor.go │ │ │ │ ├── descriptor.pb.go │ │ │ │ ├── descriptor_gostring.gen.go │ │ │ │ └── helper.go │ │ ├── golang │ │ │ └── protobuf │ │ │ │ ├── AUTHORS │ │ │ │ ├── CONTRIBUTORS │ │ │ │ ├── LICENSE │ │ │ │ └── proto │ │ │ │ ├── clone.go │ │ │ │ ├── decode.go │ │ │ │ ├── deprecated.go │ │ │ │ ├── discard.go │ │ │ │ ├── encode.go │ │ │ │ ├── equal.go │ │ │ │ ├── extensions.go │ │ │ │ ├── lib.go │ │ │ │ ├── message_set.go │ │ │ │ ├── pointer_reflect.go │ │ │ │ ├── pointer_unsafe.go │ │ │ │ ├── properties.go │ │ │ │ ├── table_marshal.go │ │ │ │ ├── table_merge.go │ │ │ │ ├── table_unmarshal.go │ │ │ │ ├── text.go │ │ │ │ └── text_parser.go │ │ ├── pmezard │ │ │ └── go-difflib │ │ │ │ ├── LICENSE │ │ │ │ └── difflib │ │ │ │ └── difflib.go │ │ └── stretchr │ │ │ └── testify │ │ │ ├── LICENSE │ │ │ └── assert │ │ │ ├── assertion_format.go │ │ │ ├── assertion_format.go.tmpl │ │ │ ├── assertion_forward.go │ │ │ ├── assertion_forward.go.tmpl │ │ │ ├── assertion_order.go │ │ │ ├── assertions.go │ │ │ ├── doc.go │ │ │ ├── errors.go │ │ │ ├── forward_assertions.go │ │ │ └── http_assertions.go │ │ ├── gopkg.in │ │ └── yaml.v2 │ │ │ ├── .travis.yml │ │ │ ├── LICENSE │ │ │ ├── LICENSE.libyaml │ │ │ ├── NOTICE │ │ │ ├── README.md │ │ │ ├── apic.go │ │ │ ├── decode.go │ │ │ ├── emitterc.go │ │ │ ├── encode.go │ │ │ ├── go.mod │ │ │ ├── parserc.go │ │ │ ├── readerc.go │ │ │ ├── resolve.go │ │ │ ├── scannerc.go │ │ │ ├── sorter.go │ │ │ ├── writerc.go │ │ │ ├── yaml.go │ │ │ ├── yamlh.go │ │ │ └── yamlprivateh.go │ │ └── modules.txt ├── java │ └── org │ │ └── dist │ │ └── patterns │ │ ├── common │ │ ├── Client.java │ │ ├── InetAddressAndPort.java │ │ ├── JsonSerDes.java │ │ ├── RequestOrResponse.java │ │ ├── SocketIO.java │ │ └── TcpListener.java │ │ ├── distlog │ │ ├── ArrayUtil.java │ │ ├── Bookie.java │ │ ├── BufferedChannel.java │ │ ├── EntryKey.java │ │ ├── EntryKeyValue.java │ │ ├── EntryLogger.java │ │ ├── FileInfo.java │ │ ├── Journal.java │ │ ├── LedgerCache.java │ │ ├── LedgerDescriptor.java │ │ ├── LedgerEntryPage.java │ │ ├── LedgerManager.java │ │ ├── LedgerMetadata.java │ │ ├── LedgerStorage.java │ │ ├── LongPairWrapper.java │ │ ├── LongWrapper.java │ │ └── SortedLedgerStorage.java │ │ ├── heartbeat │ │ └── HeartBeatScheduler.java │ │ ├── ignite │ │ └── cluster │ │ │ ├── A.java │ │ │ ├── AlwaysFalsePredicate.java │ │ │ ├── AlwaysTruePredicate.java │ │ │ ├── ClusterMetrics.java │ │ │ ├── ClusterNode.java │ │ │ ├── F.java │ │ │ ├── GridArgumentCheck.java │ │ │ ├── GridEmptyIterator.java │ │ │ ├── GridFunc.java │ │ │ ├── GridIterator.java │ │ │ ├── GridIteratorAdapter.java │ │ │ ├── GridMetadataAwareAdapter.java │ │ │ ├── GridSerializableCollection.java │ │ │ ├── GridSerializableIterator.java │ │ │ ├── HasNotEqualIdPredicate.java │ │ │ ├── IdentityClosure.java │ │ │ ├── IgniteCheckedException.java │ │ │ ├── IgniteClosure.java │ │ │ ├── IgniteException.java │ │ │ ├── IgnitePredicate.java │ │ │ ├── IgniteUtils.java │ │ │ ├── P1.java │ │ │ ├── PN.java │ │ │ ├── PredicateCollectionView.java │ │ │ ├── ServerImpl.java │ │ │ ├── TcpDiscoveryNode.java │ │ │ ├── TcpDiscoveryNodesRing.java │ │ │ ├── TcpDiscoverySpiState.java │ │ │ ├── TcpListener.java │ │ │ ├── TransformFilteringIterator.java │ │ │ ├── U.java │ │ │ └── X.java │ │ ├── imdg │ │ ├── EntryEventFactory.java │ │ ├── EntryEventImpl.java │ │ ├── InternalDataView.java │ │ ├── LocalRegion.java │ │ ├── OpType.java │ │ ├── Operation.java │ │ ├── Region.java │ │ ├── TXManagerImpl.java │ │ ├── TXStateInterface.java │ │ ├── TXStateProxy.java │ │ ├── TXStateProxyImpl.java │ │ └── TransactionId.java │ │ ├── leaderelection │ │ └── Server.java │ │ ├── pipelinedconnection │ │ └── zk │ │ │ ├── ClientCnxn.java │ │ │ ├── ClientCnxnSocket.java │ │ │ └── ClientCnxnSocketNIO.java │ │ ├── session │ │ ├── ExpiryQueue.java │ │ ├── SessionIdGenerator.java │ │ ├── SessionTracker.java │ │ └── SessionTrackerImpl.java │ │ ├── singularupdatequeue │ │ ├── Account.java │ │ ├── ExecutorBackedSingularUpdateQueue.java │ │ ├── SingleThreadedAccount.java │ │ ├── SingularUpdateQueue.java │ │ ├── SynchronizedAccount.java │ │ └── UpdateHandler.java │ │ └── wal │ │ └── WriteAheadLog.java ├── resources │ └── log4j.properties └── scala │ └── org │ └── dist │ ├── akkagossip │ ├── ClusterDaemon.scala │ ├── ClusterEvent.scala │ ├── ClusterHeartbeat.scala │ ├── DeadlineFailureDetector.scala │ ├── DefaultFailureDetectorRegistry.scala │ ├── DirectNetworkIO.scala │ ├── FailureDetectorRegistry.scala │ ├── Gossip.scala │ ├── GossipEnvelope.scala │ ├── GossipOverview.scala │ ├── Join.scala │ ├── Member.scala │ ├── MembershipState.scala │ ├── Reachability.scala │ ├── VectorClock.scala │ ├── Version.scala │ └── Welcome.scala │ ├── bookkeeper │ ├── BKException.java │ ├── BufferedChannel.java │ ├── BufferedChannelBase.java │ ├── BufferedReadChannel.java │ ├── DigestType.java │ ├── Journal.java │ ├── JournalChannel.java │ ├── LedgerEntry.java │ ├── LogMark.java │ ├── MathUtils.java │ ├── NativeIO.java │ ├── RecyclableArrayList.java │ └── ZeroBuffer.java │ ├── consensus │ └── zab │ │ ├── BinaryInputArchive.scala │ │ ├── BinaryOutputArchive.scala │ │ ├── Client.scala │ │ ├── DataTree.scala │ │ ├── FollowerHandler.scala │ │ ├── FollowerS.scala │ │ ├── FollowerZookeeperServer.scala │ │ ├── Leader.scala │ │ ├── LeaderZookeeperServer.scala │ │ ├── PrepRequestProcessor.scala │ │ ├── ProposeRequest.scala │ │ ├── QuorumPacket.scala │ │ ├── QuorumPeer.scala │ │ ├── QuorumPeerConfig.scala │ │ ├── QuorumServer.scala │ │ ├── ServerCnxn.scala │ │ ├── SynRequestProcessor.scala │ │ ├── Vote.scala │ │ └── api │ │ └── ClientRequestOrResponse.scala │ ├── disruptor │ └── SimpleDisruptor.scala │ ├── kvstore │ ├── ApplicationState.java │ ├── ArrivalWindow.scala │ ├── ColumnFamilyStore.scala │ ├── DatabaseConfiguration.scala │ ├── DbManager.scala │ ├── EndPointState.scala │ ├── FBUtilities.java │ ├── FailureDetector.scala │ ├── GossipDigest.scala │ ├── GossipDigestAck.scala │ ├── GossipDigestAck2.scala │ ├── GossipDigestSyn.scala │ ├── Gossiper.scala │ ├── GuidGenerator.java │ ├── HashingUtils.java │ ├── Header.scala │ ├── HeartBeatState.scala │ ├── IFailureDetector.java │ ├── IPartitioner.java │ ├── InetAddressAndPort.scala │ ├── JsonSerDes.scala │ ├── MD5Digest.java │ ├── Memtable.scala │ ├── Message.scala │ ├── MessagingService.scala │ ├── RandomPartitioner.java │ ├── RowMutation.scala │ ├── SequenceFile.scala │ ├── Stage.java │ ├── StorageMetadata.scala │ ├── StorageProxy.scala │ ├── StorageService.scala │ ├── Table.scala │ ├── TokenMetadata.scala │ ├── Verb.java │ ├── VersionGenerator.scala │ ├── VersionedValue.scala │ ├── client │ │ ├── Client.scala │ │ └── SocketClient.scala │ ├── locator │ │ ├── AbstractStrategy.java │ │ ├── EndPointSnitch.java │ │ ├── IEndPointSnitch.java │ │ ├── IReplicaPlacementStrategy.java │ │ └── RackUnawareStrategy.java │ └── testapp │ │ ├── GossipTestApp.scala │ │ ├── Node1.scala │ │ ├── Node2.scala │ │ ├── Node3.scala │ │ └── Utils.scala │ ├── partitioning │ └── KeyRangePartitioning.scala │ ├── patterns │ ├── failuredetector │ │ ├── ArrivalWindow.scala │ │ ├── PhiChiAccrualFailureDetector.scala │ │ ├── Sender.scala │ │ └── SimpleFailureDetector.scala │ ├── replicatedlog │ │ ├── LeaderElection.scala │ │ ├── ReplicatedKVStore.scala │ │ ├── ReplicatedWal.scala │ │ ├── Server.scala │ │ ├── TcpListener.scala │ │ ├── api │ │ │ └── VoteRequest.scala │ │ └── heartbeat │ │ │ ├── HeartBeatScheduler.scala │ │ │ └── Peer.scala │ └── wal │ │ ├── KVStore.scala │ │ ├── SetValueCommand.scala │ │ ├── Wal.scala │ │ └── WalEntry.scala │ ├── queue │ ├── admin │ │ └── CreateTopicCommand.scala │ ├── api │ │ ├── FetchRequest.scala │ │ ├── FetchResponse.scala │ │ ├── FindCoordinatorRequest.scala │ │ ├── FindCoordinatorResponse.scala │ │ ├── LeaderAndIsrRequest.scala │ │ ├── LeaderAndIsrResponse.scala │ │ ├── OffsetRequest.scala │ │ ├── ProduceRequest.scala │ │ ├── ProducerResponseStatus.scala │ │ ├── RequestOrResponse.scala │ │ ├── TopicMetadataRequest.scala │ │ ├── TopicMetadataResponse.scala │ │ ├── UpdateMetadataRequest.scala │ │ └── UpdateMetadataResponse.scala │ ├── client │ │ ├── common │ │ │ ├── BrokerPartitionInfo.scala │ │ │ ├── ClientUtils.scala │ │ │ └── PartitionAndLeader.scala │ │ ├── consumer │ │ │ ├── Consumer.scala │ │ │ └── ConsumerConfig.scala │ │ └── producer │ │ │ ├── DefaultPartitioner.scala │ │ │ ├── Practitioner.scala │ │ │ └── Producer.scala │ ├── cluster │ │ ├── Partition.scala │ │ └── Replica.scala │ ├── common │ │ ├── ErrorMapping.scala │ │ ├── FileLock.scala │ │ ├── InvalidMessageSizeException.scala │ │ ├── IteratorTemplate.scala │ │ ├── KafkaException.scala │ │ ├── KafkaScheduler.scala │ │ ├── KafkaStorageException.scala │ │ ├── KafkaZookeeperClient.scala │ │ ├── LeaderNotAvailableException.scala │ │ ├── Logging.scala │ │ ├── NoBrokersForPartitionException.scala │ │ ├── NotLeaderForPartitionException.scala │ │ ├── Pool.scala │ │ ├── ReplicaNotAvailableException.scala │ │ ├── ShutdownableThread.scala │ │ ├── StateChangeFailedException.scala │ │ ├── Topic.scala │ │ ├── TopicAndPartition.scala │ │ └── UnknownTopicOrPartitionException.scala │ ├── controller │ │ ├── AddPartitionsListener.scala │ │ ├── Controller.scala │ │ ├── ControllerBrokerRequestBatch.scala │ │ ├── ControllerChannelManager.scala │ │ ├── OfflinePartitionLeaderSelector.scala │ │ ├── PartitionLeaderSelector.scala │ │ ├── PartitionStateMachine.scala │ │ ├── ReplicaStateMachine.scala │ │ └── TopicChangeListner.scala │ ├── coordinator │ │ └── GroupMetadataManager.scala │ ├── log │ │ ├── FileMessageSet.scala │ │ ├── InvalidMessageException.scala │ │ ├── InvalidOffsetException.scala │ │ ├── KafkaTimer.scala │ │ ├── Log.scala │ │ ├── LogManager.scala │ │ ├── LogSegment.scala │ │ ├── OffsetIndex.scala │ │ ├── OffsetOutOfRangeException.scala │ │ ├── OffsetPosition.scala │ │ ├── Range.scala │ │ └── SegmentList.scala │ ├── message │ │ ├── ByteBufferBackedInputStream.scala │ │ ├── ByteBufferMessageSet.scala │ │ ├── CompressionCodec.scala │ │ ├── CompressionFactory.scala │ │ ├── KeyedMessage.scala │ │ ├── Message.scala │ │ ├── MessageAndOffset.scala │ │ ├── MessageSet.scala │ │ └── MessageSizeTooLargeException.scala │ ├── network │ │ ├── Client.scala │ │ ├── SocketClient.scala │ │ └── SocketServer.scala │ ├── server │ │ ├── Config.scala │ │ ├── FetcherThread.scala │ │ ├── HighwaterMarkCheckpoint.scala │ │ ├── KafkaApis.scala │ │ ├── KafkaZookeeper.scala │ │ ├── ReplicaManager.scala │ │ ├── Server.scala │ │ └── ZookeeperLeaderElector.scala │ └── utils │ │ ├── AdminUtils.scala │ │ ├── Utils.scala │ │ └── ZkUtils.scala │ ├── rapid │ ├── Cluster.scala │ ├── MembershipService.scala │ ├── MembershipView.scala │ ├── Paxos.scala │ ├── SocketClient.scala │ ├── SocketServer.scala │ └── messages │ │ ├── AlertMessage.scala │ │ ├── JoinMessage.scala │ │ ├── JoinResponse.scala │ │ ├── Phase1aMessage.scala │ │ ├── Phase1bMessage.scala │ │ ├── Phase2aMessage.scala │ │ ├── Phase2bMessage.scala │ │ ├── PreJoinMessage.scala │ │ ├── Proposal.scala │ │ ├── Rank.scala │ │ └── RapidMessages.scala │ ├── simplegossip │ ├── Gossiper.scala │ ├── MessagingService.scala │ ├── StorageProxy.scala │ └── StorageService.scala │ ├── simplekafka │ ├── BrokerChangeListner.scala │ ├── ConsumeRequest.scala │ ├── ConsumeResponse.scala │ ├── Controller.scala │ ├── CreateTopicCommand.scala │ ├── DelayedOperation.scala │ ├── DelayedProduce.scala │ ├── LeaderAndReplicas.scala │ ├── Partition.scala │ ├── Pool.scala │ ├── ProduceRequest.scala │ ├── ProduceResponse.java │ ├── ReplicaAssignmentStrategy.scala │ ├── ReplicaManager.scala │ ├── SequenceFile.scala │ ├── Server.scala │ ├── ShutdownableThread.scala │ ├── SimpleConsumer.scala │ ├── SimpleKafkaApi.scala │ ├── SimpleProducer.scala │ ├── SimpleSocketServer.scala │ ├── SocketClient.scala │ ├── SystemTime.java │ ├── Time.java │ ├── TimeoutException.java │ ├── Timer.java │ ├── TopicChangeHandler.scala │ ├── TopicMetadataRequest.scala │ ├── TopicMetadataResponse.scala │ ├── UpdateMetadataRequest.scala │ ├── Utils.java │ ├── ZookeeperClient.scala │ └── utils │ │ ├── KafkaThread.java │ │ └── timer │ │ ├── Time.scala │ │ ├── Timer.scala │ │ ├── TimerTask.scala │ │ ├── TimerTaskList.scala │ │ └── TimingWheel.scala │ ├── util │ ├── Networks.scala │ └── SocketIO.scala │ └── versionedkvstore │ ├── ByteUtils.java │ ├── Client.scala │ ├── ClockEntry.java │ ├── ConsistentRoutingStrategy.java │ ├── FnvHashFunction.java │ ├── HashFunction.java │ ├── InconsistencyResolver.java │ ├── InconsistentDataException.scala │ ├── InvalidClockEntryException.java │ ├── Node.scala │ ├── NodeValue.java │ ├── ObsoleteVersionException.scala │ ├── Occurred.java │ ├── ReadRepairer.java │ ├── RoutingStrategy.java │ ├── RoutingStrategyType.java │ ├── TimeBasedInconsistencyResolver.java │ ├── VectorClock.java │ ├── VectorClockInconsistencyResolver.java │ ├── VectorClockUtils.java │ ├── Version.java │ ├── Versioned.scala │ └── VersionedMap.scala └── test ├── java └── org │ └── dist │ ├── bookkeeper │ └── JournalTest.java │ ├── patterns │ ├── ignite │ │ └── cluster │ │ │ └── TcpDiscoveryNodeTest.java │ ├── imdg │ │ └── LocalRegionTest.java │ └── session │ │ ├── ExpiryQueueTest.java │ │ └── SessionTrackerImplTest.java │ └── utils │ └── JTestUtils.java └── scala └── org └── dist ├── akkagossip └── ClusterDaemonTest.scala ├── consensus ├── raft │ └── Log.scala └── zab │ ├── LeaderElectionTest.scala │ ├── LogTest.scala │ ├── QuorumPacketSerializationTest.scala │ ├── QuorumPeerTest.scala │ ├── TransactionSerializationTest.scala │ ├── ZabTestUtils.scala │ └── ZxIdTest.scala ├── kvstore ├── DbManagerTest.scala ├── GossipMessageHandlingTests.scala ├── GossipTaskTest.scala ├── GossiperTaskTest.scala ├── GossiperTest.scala ├── GossiperTestBuilder.scala ├── JsonSerDesTest.scala ├── KVStoreQuorumPutAndGetTest.scala ├── TableReadWriteTest.scala ├── TestUtils.scala ├── VersionGeneratorTest.scala └── locator │ └── RackUnawareStrategyTest.scala ├── patterns ├── common │ ├── JsonSerDesTest.java │ └── TcpListenerTest.java ├── distlog │ ├── BookieReadWriteTest.scala │ ├── LedgerCacheTest.scala │ └── LedgerMetadataTest.scala ├── failuredetector │ └── FailureDetectorTests.scala ├── leaderelection │ └── ServerTest.java ├── replicatedlog │ ├── ReplicatedKVStoreTest.scala │ ├── ServerTest.scala │ └── heartbeat │ │ └── HeartBeatSchedulerTest.scala ├── singularupdatequeue │ ├── AccountTest.java │ ├── SingleThreadedAccountCallable.java │ ├── SingleThreadedAccountPerfMain.java │ ├── SingularUpdateQueueAccountContention.java │ ├── SynchronizedAccountCallable.java │ ├── SynchronizedAccountContention.java │ └── SynchronizedAccountPerfMain.java └── wal │ ├── KVStoreTest.scala │ ├── WalTest.scala │ └── WriteAheadLogTest.java ├── queue ├── AdminUtilsTest.scala ├── ByteBufferMessageSetTest.scala ├── ControllerFailureTest.scala ├── EmbeddedZookeeper.scala ├── ProducerConsumerTest.scala ├── ProducerTest.scala ├── TestUtils.scala ├── TopicCreationTest.scala ├── ZookeeperTestHarness.scala ├── api │ └── LeaderAndIsrRequestTest.scala ├── client │ └── producer │ │ └── DefaultPartitionerTest.scala └── controller │ └── ReplicaStateMachineTest.scala ├── rapid ├── ClusterTest.scala └── PaxosTest.scala ├── simplegossip └── StorageServiceGossipTest.scala ├── simplekafka ├── BrokerChangeListenerTest.scala ├── ControllerTest.scala ├── ControllerZookeeperFailureTests.scala ├── ControllerZookeeperTest.scala ├── CreateTopicCommandTest.scala ├── CreateTopicCommandZookeeperTest.scala ├── DelayedProduceTest.scala ├── PartitionConcurrentReadWriteTest.scala ├── PartitionReadWriteTest.scala ├── ProducerConsumerTest.scala ├── ServerTest.scala ├── SimpleKafkaApiTest.scala └── TopicChangeHandlerTest.scala └── versionedkvstore └── VersionedMapTest.scala /.gitignore: -------------------------------------------------------------------------------- 1 | *.avsc 2 | *.swp 3 | checkpoint 4 | *.log 5 | ignore 6 | .vagrant 7 | logs 8 | project/project/target 9 | project/target 10 | target 11 | .idea 12 | *.iml 13 | /out 14 | .idea_modules 15 | .classpath 16 | .project 17 | /RUNNING_PID 18 | .settings 19 | .DS_Store 20 | integration/project/build.properties 21 | 22 | # hidden cross project folders 23 | shared/.js 24 | shared/.jvm 25 | 26 | 27 | # eclipse 28 | .scala_dependencies 29 | .buildpath 30 | .target 31 | bin/ 32 | journal 33 | snapshots 34 | *.sc 35 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | This is a repository with code to teach distributed systems concepts. It has some code copied from 2 | Cassandra (https://github.com/apache/cassandra) and Kafka (https://github.com/apache/kafka) 3 | repositories, and modified to make it simple to teach certain concepts. 4 | This is used only for teaching and learning purpose 5 | 6 | * Install sbt (https://www.scala-sbt.org/) 7 | * clone this repo 8 | git clone git@github.com:unmeshjoshi/distributedarchitectures.git 9 | * Run 'sbt clean test' 10 | 11 | (TODO: One of tests breaks when run with others because of some dependencies. It passes standalone) 12 | 13 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | import Settings._ 2 | 3 | val `distributedarchitectures` = project 4 | .in(file(".")) 5 | .enablePlugins(DeployApp, DockerPlugin) 6 | .settings(defaultSettings: _*) 7 | .settings( 8 | libraryDependencies ++= Dependencies.Dist, parallelExecution in Test := false 9 | ) 10 | 11 | 12 | -------------------------------------------------------------------------------- /project/Dependencies.scala: -------------------------------------------------------------------------------- 1 | import sbt._ 2 | 3 | object Dependencies { 4 | 5 | val Version = "0.1-SNAPSHOT" 6 | val Dist = Seq( 7 | Libs.`zookeeper`, 8 | Libs.`zkclient`, 9 | Libs.`yammer`, 10 | Libs.`scalaLogging`, 11 | Libs.`jacksonDatabind`, 12 | Libs.`jacksonJaxrsJsonProvider`, 13 | Libs.`jacksonJDK8Datatypes`, 14 | Libs.`jacksonDataformatCsv`, 15 | Libs.`jacksonModuleScala`, 16 | Libs.`scalaCollectionCompat`, 17 | Libs.`googleGuava`, 18 | Libs.`jamm`, 19 | Libs.`pcj`, 20 | Libs.`commons-codec`, 21 | Libs.`disruptor`, 22 | Libs.`akka-stream-typed`, 23 | Libs.`jna`, 24 | Libs.`yammer-core`, 25 | Libs.`commons-cli`, 26 | Libs.`commons-lang`, 27 | Libs.`commons-collections4`, 28 | Libs.`commons-codec`, 29 | Libs.`rocksdb`, 30 | Libs.`scalaTest` % Test, 31 | Libs.`mockito` % Test, 32 | Libs.`junit` % Test, 33 | Libs.`jmh-core` % Test, 34 | Libs.`jmh_bench` % Test, 35 | Libs.`jmh_generator_ann` % Test, 36 | Libs.`jmh_generator_bytecode` % Test 37 | ) 38 | } 39 | -------------------------------------------------------------------------------- /project/Settings.scala: -------------------------------------------------------------------------------- 1 | import sbt.Keys._ 2 | import sbt._ 3 | 4 | //noinspection TypeAnnotation 5 | // Defines the global build settings so they don't need to be edited everywhere 6 | object Settings { 7 | 8 | val buildSettings = Seq( 9 | organization := "org.http2example", 10 | organizationName := "example", 11 | organizationHomepage := Some(url("http://www.example.org")), 12 | version := Dependencies.Version, 13 | scalaVersion := Libs.ScalaVersion, 14 | crossPaths := true, 15 | parallelExecution in Test := false, 16 | fork := true 17 | ) 18 | 19 | lazy val defaultSettings = buildSettings ++ Seq( 20 | // compile options ScalaUnidoc, unidoc 21 | scalacOptions ++= Seq("-encoding", "UTF-8", "-feature", "-deprecation", "-unchecked"), 22 | javacOptions in(Compile, compile) ++= Seq("-source", "11", "-target", "11", "-Xlint:unchecked", "-Xlint:deprecation"), 23 | javaOptions in(Test, run) ++= Seq("-Djava.net.preferIPv4Stack=true") // For location service use 24 | ) 25 | } 26 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.3.3 -------------------------------------------------------------------------------- /project/project/metaplugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.timushev.sbt" % "sbt-updates" % "0.5.1") 2 | -------------------------------------------------------------------------------- /project/publishing.scala: -------------------------------------------------------------------------------- 1 | import com.typesafe.sbt.packager.archetypes.JavaAppPackaging 2 | import sbt.Keys.{name, packageBin, version} 3 | import sbt.{AutoPlugin, Plugins, Setting} 4 | object DeployApp extends AutoPlugin { 5 | import com.typesafe.sbt.packager.SettingsHelper 6 | import com.typesafe.sbt.packager.universal.UniversalPlugin 7 | import UniversalPlugin.autoImport.{Universal, UniversalDocs} 8 | import sbtbuildinfo.BuildInfoPlugin 9 | import BuildInfoPlugin.autoImport._ 10 | 11 | override def requires: Plugins = UniversalPlugin && JavaAppPackaging && BuildInfoPlugin 12 | 13 | override def projectSettings: Seq[Setting[_]] = 14 | SettingsHelper.makeDeploymentSettings(Universal, packageBin in Universal, "zip") ++ 15 | SettingsHelper.makeDeploymentSettings(UniversalDocs, packageBin in UniversalDocs, "zip") ++ Seq( 16 | buildInfoKeys := Seq[BuildInfoKey](name, version), 17 | buildInfoPackage := "micro.services" 18 | ) 19 | } 20 | -------------------------------------------------------------------------------- /src/main/go/README.md: -------------------------------------------------------------------------------- 1 | * Download protoc from https://github.com/protocolbuffers/protobuf/releases 2 | * Extract and add it to path 3 | 4 | e.g. ```unzip -x ~/Downloads/protoc-3.11.4-linux-aarch_64.zip -d ./tmp/``` 5 | ```export PATH=./tmp/bin:$PATH``` 6 | * Get gogoproto source and binary 7 | 8 | ```GO111MODULE=off go get -u -d github.com/gogo/protobuf``` 9 | * cd newraft/raftpb 10 | * Run 11 | 12 | ```protoc -I=. -I=$GOPATH/src -I=$GOPATH/src/github.com/gogo/protobuf --gogo_out=. ./*.proto``` -------------------------------------------------------------------------------- /src/main/go/consensus: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/unmeshjoshi/distributedarchitectures/b0ebe2adef49ca27ff54ea2d94a8504914d45580/src/main/go/consensus -------------------------------------------------------------------------------- /src/main/go/go.mod: -------------------------------------------------------------------------------- 1 | module consensus 2 | 3 | require ( 4 | github.com/cockroachdb/datadriven v0.0.0-20191203163246-053dcac06c27 5 | github.com/coreos/etcd v2.3.8+incompatible // indirect 6 | github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7 // indirect 7 | github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf 8 | github.com/gogo/protobuf v1.3.1 9 | github.com/golang/protobuf v1.4.3 10 | github.com/google/btree v1.0.0 11 | github.com/prometheus/client_golang v1.4.0 12 | github.com/prometheus/common v0.9.1 13 | github.com/stretchr/testify v1.4.0 14 | go.etcd.io/bbolt v1.3.5 15 | go.etcd.io/etcd v2.3.8+incompatible 16 | go.uber.org/zap v1.13.0 17 | google.golang.org/protobuf v1.25.0 // indirect 18 | 19 | ) 20 | 21 | go 1.13 22 | -------------------------------------------------------------------------------- /src/main/go/kvstore/embed/Config.go: -------------------------------------------------------------------------------- 1 | package embed 2 | 3 | import "go.uber.org/zap" 4 | 5 | type Config struct { 6 | TickMs int 7 | ElectionMs int 8 | Logger *zap.Logger 9 | } 10 | 11 | 12 | // NewConfig creates a new Config populated with default values. 13 | func NewConfig() *Config { 14 | cfg := &Config{ 15 | Logger: zap.NewExample(), 16 | TickMs: 100, 17 | ElectionMs: 1000, 18 | } 19 | return cfg 20 | } -------------------------------------------------------------------------------- /src/main/go/kvstore/runtime/fds_linux.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // Package runtime implements utility functions for runtime systems. 16 | package runtime 17 | 18 | import ( 19 | "io/ioutil" 20 | "syscall" 21 | ) 22 | 23 | func FDLimit() (uint64, error) { 24 | var rlimit syscall.Rlimit 25 | if err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &rlimit); err != nil { 26 | return 0, err 27 | } 28 | return rlimit.Cur, nil 29 | } 30 | 31 | func FDUsage() (uint64, error) { 32 | fds, err := ioutil.ReadDir("/proc/self/fd") 33 | if err != nil { 34 | return 0, err 35 | } 36 | return uint64(len(fds)), nil 37 | } 38 | -------------------------------------------------------------------------------- /src/main/go/kvstore/server_test.go: -------------------------------------------------------------------------------- 1 | package kvstore 2 | 3 | import "testing" 4 | 5 | func TestServer(t *testing.T) { 6 | server, err := NewServer() 7 | if (err == nil) { 8 | server.run() 9 | } 10 | } -------------------------------------------------------------------------------- /src/main/go/kvstore/storage.go: -------------------------------------------------------------------------------- 1 | package kvstore 2 | 3 | import ( 4 | "consensus/newraft/raftpb" 5 | pb "consensus/newraft/raftpb" 6 | "consensus/newraft/wal" 7 | "fmt" 8 | ) 9 | 10 | type Storage interface { 11 | // Save function saves ents and state to the underlying stable storage. 12 | // Save MUST block until st and ents are on stable storage. 13 | Save(st raftpb.HardState, ents []raftpb.Entry) error 14 | // SaveSnap function saves snapshot to the underlying stable storage. 15 | SaveSnap(pb.Snapshot) error 16 | // Close closes the Storage and performs finalization. 17 | Close() error 18 | } 19 | 20 | func NewStorage(w *wal.WAL, s *Snapshotter) Storage { 21 | return &storage{w, s} 22 | } 23 | 24 | type storage struct { 25 | *wal.WAL 26 | *Snapshotter 27 | } 28 | 29 | func (st *storage) SaveSnap(pb.Snapshot) error { 30 | fmt.Printf("Savesnap called on (%v)", st) 31 | return nil 32 | } 33 | 34 | 35 | type Snapshotter struct { 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/go/kvstore/types/id.go: -------------------------------------------------------------------------------- 1 | package types 2 | 3 | import "strconv" 4 | 5 | // ID represents a generic identifier which is canonically 6 | // stored as a uint64 but is typically represented as a 7 | // base-16 string for input/output 8 | type ID uint64 9 | 10 | func (i ID) String() string { 11 | return strconv.FormatUint(uint64(i), 16) 12 | } 13 | 14 | // IDFromString attempts to create an ID from a base-16 string. 15 | func IDFromString(s string) (ID, error) { 16 | i, err := strconv.ParseUint(s, 16, 64) 17 | return ID(i), err 18 | } 19 | 20 | // IDSlice implements the sort interface 21 | type IDSlice []ID 22 | 23 | func (p IDSlice) Len() int { return len(p) } 24 | func (p IDSlice) Less(i, j int) bool { return uint64(p[i]) < uint64(p[j]) } 25 | func (p IDSlice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } 26 | -------------------------------------------------------------------------------- /src/main/go/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | func main() { 4 | index 5 | } 6 | -------------------------------------------------------------------------------- /src/main/go/mvcc/store.go: -------------------------------------------------------------------------------- 1 | package mvcc 2 | 3 | import bolt "go.etcd.io/bbolt" 4 | 5 | type store struct { 6 | kvindex index 7 | db * bolt.DB 8 | } 9 | 10 | func (s *store) put() { 11 | db.Begin() 12 | } -------------------------------------------------------------------------------- /src/main/go/newraft/README.md: -------------------------------------------------------------------------------- 1 | Download protoc-3.11.2-linux-x86_64.zip from https://github.com/protocolbuffers/protobuf/releases 2 | extract to say ~/softwares/protoc and add to PATH 3 | 4 | export PATH=~/softwares/protoc/bin:$PATH 5 | go get github.com/gogo/protobuf/gogoproto 6 | 7 | export PATH=$GOPATH/bin:$PATH 8 | 9 | cd /consensus/newraft/raftpb 10 | protoc -I=. -I=$GOPATH/src -I=$GOPATH/src/github.com/gogo/protobuf --gogo_out=. ./*.proto -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/joint_autoleave.txt: -------------------------------------------------------------------------------- 1 | # Test the autoleave argument to EnterJoint. It defaults to false in the 2 | # datadriven tests. The flag has no associated semantics in this package, 3 | # it is simply passed through. 4 | simple 5 | v1 6 | ---- 7 | voters=(1) 8 | 1: StateProbe match=0 next=0 9 | 10 | # Autoleave is reflected in the config. 11 | enter-joint autoleave=true 12 | v2 v3 13 | ---- 14 | voters=(1 2 3)&&(1) autoleave 15 | 1: StateProbe match=0 next=0 16 | 2: StateProbe match=0 next=1 17 | 3: StateProbe match=0 next=1 18 | 19 | # Can't enter-joint twice, even if autoleave changes. 20 | enter-joint autoleave=false 21 | ---- 22 | config is already joint 23 | 24 | leave-joint 25 | ---- 26 | voters=(1 2 3) 27 | 1: StateProbe match=0 next=0 28 | 2: StateProbe match=0 next=1 29 | 3: StateProbe match=0 next=1 30 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/joint_idempotency.txt: -------------------------------------------------------------------------------- 1 | # Verify that operations upon entering the joint state are idempotent, i.e. 2 | # removing an absent node is fine, etc. 3 | 4 | simple 5 | v1 6 | ---- 7 | voters=(1) 8 | 1: StateProbe match=0 next=0 9 | 10 | enter-joint 11 | r1 r2 r9 v2 v3 v4 v2 v3 v4 l2 l2 r4 r4 l1 l1 12 | ---- 13 | voters=(3)&&(1) learners=(2) learners_next=(1) 14 | 1: StateProbe match=0 next=0 15 | 2: StateProbe match=0 next=1 learner 16 | 3: StateProbe match=0 next=1 17 | 18 | leave-joint 19 | ---- 20 | voters=(3) learners=(1 2) 21 | 1: StateProbe match=0 next=0 learner 22 | 2: StateProbe match=0 next=1 learner 23 | 3: StateProbe match=0 next=1 24 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/joint_learners_next.txt: -------------------------------------------------------------------------------- 1 | # Verify that when a voter is demoted in a joint config, it will show up in 2 | # learners_next until the joint config is left, and only then will the progress 3 | # turn into that of a learner, without resetting the progress. Note that this 4 | # last fact is verified by `next`, which can tell us which "round" the progress 5 | # was originally created in. 6 | 7 | simple 8 | v1 9 | ---- 10 | voters=(1) 11 | 1: StateProbe match=0 next=0 12 | 13 | enter-joint 14 | v2 l1 15 | ---- 16 | voters=(2)&&(1) learners_next=(1) 17 | 1: StateProbe match=0 next=0 18 | 2: StateProbe match=0 next=1 19 | 20 | leave-joint 21 | ---- 22 | voters=(2) learners=(1) 23 | 1: StateProbe match=0 next=0 learner 24 | 2: StateProbe match=0 next=1 25 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/simple_idempotency.txt: -------------------------------------------------------------------------------- 1 | simple 2 | v1 3 | ---- 4 | voters=(1) 5 | 1: StateProbe match=0 next=0 6 | 7 | simple 8 | v1 9 | ---- 10 | voters=(1) 11 | 1: StateProbe match=0 next=0 12 | 13 | simple 14 | v2 15 | ---- 16 | voters=(1 2) 17 | 1: StateProbe match=0 next=0 18 | 2: StateProbe match=0 next=2 19 | 20 | simple 21 | l1 22 | ---- 23 | voters=(2) learners=(1) 24 | 1: StateProbe match=0 next=0 learner 25 | 2: StateProbe match=0 next=2 26 | 27 | simple 28 | l1 29 | ---- 30 | voters=(2) learners=(1) 31 | 1: StateProbe match=0 next=0 learner 32 | 2: StateProbe match=0 next=2 33 | 34 | simple 35 | r1 36 | ---- 37 | voters=(2) 38 | 2: StateProbe match=0 next=2 39 | 40 | simple 41 | r1 42 | ---- 43 | voters=(2) 44 | 2: StateProbe match=0 next=2 45 | 46 | simple 47 | v3 48 | ---- 49 | voters=(2 3) 50 | 2: StateProbe match=0 next=2 51 | 3: StateProbe match=0 next=7 52 | 53 | simple 54 | r3 55 | ---- 56 | voters=(2) 57 | 2: StateProbe match=0 next=2 58 | 59 | simple 60 | r3 61 | ---- 62 | voters=(2) 63 | 2: StateProbe match=0 next=2 64 | 65 | simple 66 | r4 67 | ---- 68 | voters=(2) 69 | 2: StateProbe match=0 next=2 70 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/simple_promote_demote.txt: -------------------------------------------------------------------------------- 1 | # Set up three voters for this test. 2 | 3 | simple 4 | v1 5 | ---- 6 | voters=(1) 7 | 1: StateProbe match=0 next=0 8 | 9 | simple 10 | v2 11 | ---- 12 | voters=(1 2) 13 | 1: StateProbe match=0 next=0 14 | 2: StateProbe match=0 next=1 15 | 16 | simple 17 | v3 18 | ---- 19 | voters=(1 2 3) 20 | 1: StateProbe match=0 next=0 21 | 2: StateProbe match=0 next=1 22 | 3: StateProbe match=0 next=2 23 | 24 | # Can atomically demote and promote without a hitch. 25 | # This is pointless, but possible. 26 | simple 27 | l1 v1 28 | ---- 29 | voters=(1 2 3) 30 | 1: StateProbe match=0 next=0 31 | 2: StateProbe match=0 next=1 32 | 3: StateProbe match=0 next=2 33 | 34 | # Can demote a voter. 35 | simple 36 | l2 37 | ---- 38 | voters=(1 3) learners=(2) 39 | 1: StateProbe match=0 next=0 40 | 2: StateProbe match=0 next=1 learner 41 | 3: StateProbe match=0 next=2 42 | 43 | # Can atomically promote and demote the same voter. 44 | # This is pointless, but possible. 45 | simple 46 | v2 l2 47 | ---- 48 | voters=(1 3) learners=(2) 49 | 1: StateProbe match=0 next=0 50 | 2: StateProbe match=0 next=1 learner 51 | 3: StateProbe match=0 next=2 52 | 53 | # Can promote a voter. 54 | simple 55 | v2 56 | ---- 57 | voters=(1 2 3) 58 | 1: StateProbe match=0 next=0 59 | 2: StateProbe match=0 next=1 60 | 3: StateProbe match=0 next=2 61 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/simple_safety.txt: -------------------------------------------------------------------------------- 1 | simple 2 | l1 3 | ---- 4 | removed all voters 5 | 6 | simple 7 | v1 8 | ---- 9 | voters=(1) 10 | 1: StateProbe match=0 next=1 11 | 12 | simple 13 | v2 l3 14 | ---- 15 | voters=(1 2) learners=(3) 16 | 1: StateProbe match=0 next=1 17 | 2: StateProbe match=0 next=2 18 | 3: StateProbe match=0 next=2 learner 19 | 20 | simple 21 | r1 v5 22 | ---- 23 | more than one voter changed without entering joint config 24 | 25 | simple 26 | r1 r2 27 | ---- 28 | removed all voters 29 | 30 | simple 31 | v3 v4 32 | ---- 33 | more than one voter changed without entering joint config 34 | 35 | simple 36 | l1 v5 37 | ---- 38 | more than one voter changed without entering joint config 39 | 40 | simple 41 | l1 l2 42 | ---- 43 | removed all voters 44 | 45 | simple 46 | l2 l3 l4 l5 47 | ---- 48 | voters=(1) learners=(2 3 4 5) 49 | 1: StateProbe match=0 next=1 50 | 2: StateProbe match=0 next=2 learner 51 | 3: StateProbe match=0 next=2 learner 52 | 4: StateProbe match=0 next=8 learner 53 | 5: StateProbe match=0 next=8 learner 54 | 55 | simple 56 | r1 57 | ---- 58 | removed all voters 59 | 60 | simple 61 | r2 r3 r4 r5 62 | ---- 63 | voters=(1) 64 | 1: StateProbe match=0 next=1 65 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/update.txt: -------------------------------------------------------------------------------- 1 | # Nobody cares about ConfChangeUpdateNode, but at least use it once. It is used 2 | # by etcd as a convenient way to pass a blob through their conf change machinery 3 | # that updates information tracked outside of raft. 4 | 5 | simple 6 | v1 7 | ---- 8 | voters=(1) 9 | 1: StateProbe match=0 next=0 10 | 11 | simple 12 | v2 u1 13 | ---- 14 | voters=(1 2) 15 | 1: StateProbe match=0 next=0 16 | 2: StateProbe match=0 next=1 17 | 18 | simple 19 | u1 u2 u3 u1 u2 u3 20 | ---- 21 | voters=(1 2) 22 | 1: StateProbe match=0 next=0 23 | 2: StateProbe match=0 next=1 24 | -------------------------------------------------------------------------------- /src/main/go/newraft/confchange/testdata/zero.txt: -------------------------------------------------------------------------------- 1 | # NodeID zero is ignored. 2 | simple 3 | v1 r0 v0 l0 4 | ---- 5 | voters=(1) 6 | 1: StateProbe match=0 next=0 7 | -------------------------------------------------------------------------------- /src/main/go/newraft/crc/crc.go: -------------------------------------------------------------------------------- 1 | // Copyright 2009 The Go Authors. All rights reserved. 2 | // Use of this source code is governed by a BSD-style 3 | // license that can be found in the LICENSE file. 4 | 5 | // Package crc provides utility function for cyclic redundancy check 6 | // algorithms. 7 | package crc 8 | 9 | import ( 10 | "hash" 11 | "hash/crc32" 12 | ) 13 | 14 | // The size of a CRC-32 checksum in bytes. 15 | const Size = 4 16 | 17 | type digest struct { 18 | crc uint32 19 | tab *crc32.Table 20 | } 21 | 22 | // New creates a new hash.Hash32 computing the CRC-32 checksum 23 | // using the polynomial represented by the Table. 24 | // Modified by xiangli to take a prevcrc. 25 | func New(prev uint32, tab *crc32.Table) hash.Hash32 { return &digest{prev, tab} } 26 | 27 | func (d *digest) Size() int { return Size } 28 | 29 | func (d *digest) BlockSize() int { return 1 } 30 | 31 | func (d *digest) Reset() { d.crc = 0 } 32 | 33 | func (d *digest) Write(p []byte) (n int, err error) { 34 | d.crc = crc32.Update(d.crc, d.tab, p) 35 | return len(p), nil 36 | } 37 | 38 | func (d *digest) Sum32() uint32 { return d.crc } 39 | 40 | func (d *digest) Sum(in []byte) []byte { 41 | s := d.Sum32() 42 | return append(in, byte(s>>24), byte(s>>16), byte(s>>8), byte(s)) 43 | } 44 | -------------------------------------------------------------------------------- /src/main/go/newraft/example/raft.go: -------------------------------------------------------------------------------- 1 | package example 2 | 3 | import ( 4 | "consensus/newraft" 5 | "consensus/newraft/raftpb" 6 | wal "consensus/newraft/wal" 7 | ) 8 | 9 | // TickMs: 100, 10 | // ElectionMs: 1000, 11 | // A key-value stream backed by raft 12 | type raftNode struct { 13 | proposeC <-chan string // proposed messages (k,v) 14 | confChangeC <-chan raftpb.ConfChange // proposed cluster config changes 15 | commitC chan<- *string // entries committed to log (k,v) 16 | errorC chan<- error // errors from raft session 17 | 18 | id int // client ID for raft session 19 | peers []string // raft peer URLs 20 | join bool // node is joining an existing cluster 21 | waldir string // path to WAL directory 22 | snapdir string // path to snapshot directory 23 | getSnapshot func() ([]byte, error) 24 | lastIndex uint64 // index of log at start 25 | 26 | confState raftpb.ConfState 27 | snapshotIndex uint64 28 | appliedIndex uint64 29 | 30 | // raft backing for the commit/error channel 31 | node newraft.Node 32 | raftStorage *newraft.MemoryStorage 33 | wal *wal.WAL 34 | } 35 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/dir_unix.go: -------------------------------------------------------------------------------- 1 | // Copyright 2016 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // +build !windows 16 | 17 | package fileutil 18 | 19 | import "os" 20 | 21 | // OpenDir opens a directory for syncing. 22 | func OpenDir(path string) (*os.File, error) { return os.Open(path) } 23 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/doc.go: -------------------------------------------------------------------------------- 1 | // Copyright 2018 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // Package fileutil implements utility functions related to files and paths. 16 | package fileutil 17 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/lock.go: -------------------------------------------------------------------------------- 1 | // Copyright 2016 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package fileutil 16 | 17 | import ( 18 | "errors" 19 | "os" 20 | ) 21 | 22 | var ( 23 | ErrLocked = errors.New("fileutil: file already locked") 24 | ) 25 | 26 | type LockedFile struct{ *os.File } 27 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/lock_flock.go: -------------------------------------------------------------------------------- 1 | // Copyright 2016 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // +build !windows,!plan9,!solaris 16 | 17 | package fileutil 18 | 19 | import ( 20 | "os" 21 | "syscall" 22 | ) 23 | 24 | func flockTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { 25 | f, err := os.OpenFile(path, flag, perm) 26 | if err != nil { 27 | return nil, err 28 | } 29 | if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX|syscall.LOCK_NB); err != nil { 30 | f.Close() 31 | if err == syscall.EWOULDBLOCK { 32 | err = ErrLocked 33 | } 34 | return nil, err 35 | } 36 | return &LockedFile{f}, nil 37 | } 38 | 39 | func flockLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) { 40 | f, err := os.OpenFile(path, flag, perm) 41 | if err != nil { 42 | return nil, err 43 | } 44 | if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX); err != nil { 45 | f.Close() 46 | return nil, err 47 | } 48 | return &LockedFile{f}, err 49 | } 50 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/sync.go: -------------------------------------------------------------------------------- 1 | // Copyright 2016 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // +build !linux,!darwin 16 | 17 | package fileutil 18 | 19 | import "os" 20 | 21 | // Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform. 22 | func Fsync(f *os.File) error { 23 | return f.Sync() 24 | } 25 | 26 | // Fdatasync is a wrapper around file.Sync(). Special handling is needed on linux platform. 27 | func Fdatasync(f *os.File) error { 28 | return f.Sync() 29 | } 30 | -------------------------------------------------------------------------------- /src/main/go/newraft/fileutil/sync_linux.go: -------------------------------------------------------------------------------- 1 | // Copyright 2016 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // +build linux 16 | 17 | package fileutil 18 | 19 | import ( 20 | "os" 21 | "syscall" 22 | ) 23 | 24 | // Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform. 25 | func Fsync(f *os.File) error { 26 | return f.Sync() 27 | } 28 | 29 | // Fdatasync is similar to fsync(), but does not flush modified metadata 30 | // unless that metadata is needed in order to allow a subsequent data retrieval 31 | // to be correctly handled. 32 | func Fdatasync(f *os.File) error { 33 | return syscall.Fdatasync(int(f.Fd())) 34 | } 35 | -------------------------------------------------------------------------------- /src/main/go/newraft/ioutil/reader.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | // Package ioutil implements I/O utility functions. 16 | package ioutil 17 | 18 | import "io" 19 | 20 | // NewLimitedBufferReader returns a reader that reads from the given reader 21 | // but limits the amount of data returned to at most n bytes. 22 | func NewLimitedBufferReader(r io.Reader, n int) io.Reader { 23 | return &limitedBufferReader{ 24 | r: r, 25 | n: n, 26 | } 27 | } 28 | 29 | type limitedBufferReader struct { 30 | r io.Reader 31 | n int 32 | } 33 | 34 | func (r *limitedBufferReader) Read(p []byte) (n int, err error) { 35 | np := p 36 | if len(np) > r.n { 37 | np = np[:r.n] 38 | } 39 | return r.r.Read(np) 40 | } 41 | -------------------------------------------------------------------------------- /src/main/go/newraft/ioutil/reader_test.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package ioutil 16 | 17 | import ( 18 | "bytes" 19 | "testing" 20 | ) 21 | 22 | func TestLimitedBufferReaderRead(t *testing.T) { 23 | buf := bytes.NewBuffer(make([]byte, 10)) 24 | ln := 1 25 | lr := NewLimitedBufferReader(buf, ln) 26 | n, err := lr.Read(make([]byte, 10)) 27 | if err != nil { 28 | t.Fatalf("unexpected read error: %v", err) 29 | } 30 | if n != ln { 31 | t.Errorf("len(data read) = %d, want %d", n, ln) 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/go/newraft/ioutil/util.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package ioutil 16 | 17 | import ( 18 | "io" 19 | "os" 20 | ) 21 | 22 | // WriteAndSyncFile behaves just like ioutil.WriteFile in the standard library, 23 | // but calls Sync before closing the file. WriteAndSyncFile guarantees the data 24 | // is synced if there is no error returned. 25 | func WriteAndSyncFile(filename string, data []byte, perm os.FileMode) error { 26 | f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm) 27 | if err != nil { 28 | return err 29 | } 30 | n, err := f.Write(data) 31 | if err == nil && n < len(data) { 32 | err = io.ErrShortWrite 33 | } 34 | if err == nil { 35 | err = f.Sync() 36 | } 37 | if err1 := f.Close(); err == nil { 38 | err = err1 39 | } 40 | return err 41 | } 42 | -------------------------------------------------------------------------------- /src/main/go/newraft/quorum/bench_test.go: -------------------------------------------------------------------------------- 1 | // Copyright 2019 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package quorum 16 | 17 | import ( 18 | "fmt" 19 | "math" 20 | "math/rand" 21 | "testing" 22 | ) 23 | 24 | func BenchmarkMajorityConfig_CommittedIndex(b *testing.B) { 25 | // go test -run - -bench . -benchmem ./raft/quorum 26 | for _, n := range []int{1, 3, 5, 7, 9, 11} { 27 | b.Run(fmt.Sprintf("voters=%d", n), func(b *testing.B) { 28 | c := MajorityConfig{} 29 | l := mapAckIndexer{} 30 | for i := uint64(0); i < uint64(n); i++ { 31 | c[i+1] = struct{}{} 32 | l[i+1] = Index(rand.Int63n(math.MaxInt64)) 33 | } 34 | 35 | for i := 0; i < b.N; i++ { 36 | _ = c.CommittedIndex(l) 37 | } 38 | }) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/go/newraft/quorum/voteresult_string.go: -------------------------------------------------------------------------------- 1 | // Code generated by "stringer -type=VoteResult"; DO NOT EDIT. 2 | 3 | package quorum 4 | 5 | import "strconv" 6 | 7 | func _() { 8 | // An "invalid array index" compiler error signifies that the constant values have changed. 9 | // Re-run the stringer command to generate them again. 10 | var x [1]struct{} 11 | _ = x[VotePending-1] 12 | _ = x[VoteLost-2] 13 | _ = x[VoteWon-3] 14 | } 15 | 16 | const _VoteResult_name = "VotePendingVoteLostVoteWon" 17 | 18 | var _VoteResult_index = [...]uint8{0, 11, 19, 26} 19 | 20 | func (i VoteResult) String() string { 21 | i -= 1 22 | if i >= VoteResult(len(_VoteResult_index)-1) { 23 | return "VoteResult(" + strconv.FormatInt(int64(i+1), 10) + ")" 24 | } 25 | return _VoteResult_name[_VoteResult_index[i]:_VoteResult_index[i+1]] 26 | } 27 | -------------------------------------------------------------------------------- /src/main/go/newraft/raftpb/confstate.go: -------------------------------------------------------------------------------- 1 | // Copyright 2019 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package raftpb 16 | 17 | import ( 18 | "fmt" 19 | "reflect" 20 | "sort" 21 | ) 22 | 23 | // Equivalent returns a nil error if the inputs describe the same configuration. 24 | // On mismatch, returns a descriptive error showing the differences. 25 | func (cs ConfState) Equivalent(cs2 ConfState) error { 26 | cs1 := cs 27 | orig1, orig2 := cs1, cs2 28 | s := func(sl *[]uint64) { 29 | *sl = append([]uint64(nil), *sl...) 30 | sort.Slice(*sl, func(i, j int) bool { return (*sl)[i] < (*sl)[j] }) 31 | } 32 | 33 | for _, cs := range []*ConfState{&cs1, &cs2} { 34 | s(&cs.Voters) 35 | s(&cs.Learners) 36 | s(&cs.VotersOutgoing) 37 | s(&cs.LearnersNext) 38 | cs.XXX_unrecognized = nil 39 | } 40 | 41 | if !reflect.DeepEqual(cs1, cs2) { 42 | return fmt.Errorf("ConfStates not equivalent after sorting:\n%+#v\n%+#v\nInputs were:\n%+#v\n%+#v", cs1, cs2, orig1, orig2) 43 | } 44 | return nil 45 | } 46 | -------------------------------------------------------------------------------- /src/main/go/newraft/wal/walpb/record.go: -------------------------------------------------------------------------------- 1 | // Copyright 2015 The etcd Authors 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); 4 | // you may not use this file except in compliance with the License. 5 | // You may obtain a copy of the License at 6 | // 7 | // http://www.apache.org/licenses/LICENSE-2.0 8 | // 9 | // Unless required by applicable law or agreed to in writing, software 10 | // distributed under the License is distributed on an "AS IS" BASIS, 11 | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | // See the License for the specific language governing permissions and 13 | // limitations under the License. 14 | 15 | package walpb 16 | 17 | import "errors" 18 | 19 | var ( 20 | ErrCRCMismatch = errors.New("walpb: crc mismatch") 21 | ) 22 | 23 | func (rec *Record) Validate(crc uint32) error { 24 | if rec.Crc == crc { 25 | return nil 26 | } 27 | rec.Reset() 28 | return ErrCRCMismatch 29 | } 30 | -------------------------------------------------------------------------------- /src/main/go/newraft/wal/walpb/record.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | package walpb; 3 | 4 | import "gogoproto/gogo.proto"; 5 | 6 | option (gogoproto.marshaler_all) = true; 7 | option (gogoproto.sizer_all) = true; 8 | option (gogoproto.unmarshaler_all) = true; 9 | option (gogoproto.goproto_getters_all) = false; 10 | 11 | message Record { 12 | optional int64 type = 1 [(gogoproto.nullable) = false]; 13 | optional uint32 crc = 2 [(gogoproto.nullable) = false]; 14 | optional bytes data = 3; 15 | } 16 | 17 | message Snapshot { 18 | optional uint64 index = 1 [(gogoproto.nullable) = false]; 19 | optional uint64 term = 2 [(gogoproto.nullable) = false]; 20 | } 21 | -------------------------------------------------------------------------------- /src/main/go/raft/log_entry_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import ( 4 | "bytes" 5 | "encoding/json" 6 | "fmt" 7 | "io/ioutil" 8 | "testing" 9 | ) 10 | 11 | func TestLogRecovery(t *testing.T) { 12 | tmpLog := newLog() 13 | e0, _ := newLogEntry(tmpLog, nil, 1, 1, &testCommand{X: "foo"}) 14 | e1, _ := newLogEntry(tmpLog, nil, 2, 1, &testCommand{X: "bar"}) 15 | f, _ := ioutil.TempFile("", "raft-log-") 16 | 17 | e0.WriteTo(f) 18 | e1.WriteTo(f) 19 | f.WriteString("CORRUPT!") 20 | f.Close() 21 | 22 | log := newLog() 23 | 24 | if err := log.open(f.Name()); err != nil { 25 | t.Fatalf("Unable to open log: %v", err) 26 | } 27 | 28 | entry := log.entries[0] 29 | r := bytes.NewReader(entry.pb.Command) 30 | c := testCommand{} 31 | err := json.NewDecoder(r).Decode(&c) 32 | if (err != nil) { 33 | t.Fail() 34 | } 35 | fmt.Println(c) 36 | } -------------------------------------------------------------------------------- /src/main/go/raft/raftpb/append_entries_request.proto: -------------------------------------------------------------------------------- 1 | package raftpb; 2 | 3 | import "log_entry.proto"; 4 | 5 | message AppendEntriesRequest { 6 | required uint64 Term=1; 7 | required uint64 PrevLogIndex=2; 8 | required uint64 PrevLogTerm=3; 9 | required uint64 CommitIndex=4; 10 | required string LeaderName=5; 11 | repeated LogEntry Entries=6; 12 | } -------------------------------------------------------------------------------- /src/main/go/raft/raftpb/append_entries_responses.proto: -------------------------------------------------------------------------------- 1 | package raftpb; 2 | 3 | message AppendEntriesResponse { 4 | required uint64 Term=1; 5 | required uint64 Index=2; 6 | required uint64 CommitIndex=3; 7 | required bool Success=4; 8 | } -------------------------------------------------------------------------------- /src/main/go/raft/raftpb/log_entry.proto: -------------------------------------------------------------------------------- 1 | package raftpb; 2 | 3 | message LogEntry { 4 | required uint64 Index=1; 5 | required uint64 Term=2; 6 | required string CommandName=3; 7 | optional bytes Command=4; // for nop-command 8 | } -------------------------------------------------------------------------------- /src/main/go/raft/raftpb/request_vote_request.proto: -------------------------------------------------------------------------------- 1 | package raftpb; 2 | 3 | message RequestVoteRequest { 4 | required uint64 Term=1; 5 | required uint64 LastLogIndex=2; 6 | required uint64 LastLogTerm=3; 7 | required string CandidateName=4; 8 | } -------------------------------------------------------------------------------- /src/main/go/raft/raftpb/request_vote_responses.proto: -------------------------------------------------------------------------------- 1 | package raftpb; 2 | 3 | message RequestVoteResponse { 4 | required uint64 Term=1; 5 | required bool VoteGranted=2; 6 | } -------------------------------------------------------------------------------- /src/main/go/raft/storage.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import "fmt" 4 | 5 | type Storage interface { 6 | // Save function saves ents and state to the underlying stable storage. 7 | // Save MUST block until st and ents are on stable storage. 8 | Save(st string, ents []uint) error 9 | // SaveSnap function saves snapshot to the underlying stable storage. 10 | SaveSnap(snap string) error 11 | // Close closes the Storage and performs finalization. 12 | Close() error 13 | } 14 | 15 | func NewStorage(w *WAL, s *Snapshotter) Storage { 16 | return &storage{w, s} 17 | } 18 | 19 | type storage struct { 20 | *WAL 21 | *Snapshotter 22 | } 23 | 24 | func (st *storage) SaveSnap(snap string) error { 25 | fmt.Printf("Savesnap called on (%v)", st) 26 | return nil 27 | } 28 | 29 | type WAL struct { 30 | } 31 | 32 | func (wl *WAL) Close() error { 33 | fmt.Printf("Close called on (%v)", wl) 34 | return nil 35 | } 36 | 37 | func (wl *WAL) Save(st string, ents []uint) error { 38 | fmt.Printf("Save called on (%v)", wl) 39 | return nil 40 | } 41 | 42 | type Snapshotter struct { 43 | 44 | } 45 | -------------------------------------------------------------------------------- /src/main/go/raft/storage_test.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | import "testing" 4 | 5 | func TestStorage(t *testing.T) { 6 | var newStorage Storage = NewStorage(&WAL{}, &Snapshotter{}) 7 | uints := []uint{2, 3, 5, 7, 11, 13} 8 | newStorage.Save("test", uints) 9 | newStorage.SaveSnap("snap") 10 | newStorage.Close() 11 | } -------------------------------------------------------------------------------- /src/main/go/raft/transporter.go: -------------------------------------------------------------------------------- 1 | package raft 2 | 3 | type Transporter interface { 4 | //SendVoteRequest(server server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse 5 | SendAppendEntriesRequest(server *server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse 6 | 7 | } 8 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/davecgh/go-spew/LICENSE: -------------------------------------------------------------------------------- 1 | ISC License 2 | 3 | Copyright (c) 2012-2016 Dave Collins 4 | 5 | Permission to use, copy, modify, and distribute this software for any 6 | purpose with or without fee is hereby granted, provided that the above 7 | copyright notice and this permission notice appear in all copies. 8 | 9 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES 10 | WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF 11 | MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR 12 | ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES 13 | WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN 14 | ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF 15 | OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. 16 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/gogo/protobuf/AUTHORS: -------------------------------------------------------------------------------- 1 | # This is the official list of GoGo authors for copyright purposes. 2 | # This file is distinct from the CONTRIBUTORS file, which 3 | # lists people. For example, employees are listed in CONTRIBUTORS, 4 | # but not in AUTHORS, because the employer holds the copyright. 5 | 6 | # Names should be added to this file as one of 7 | # Organization's name 8 | # Individual's name 9 | # Individual's name 10 | 11 | # Please keep the list sorted. 12 | 13 | Sendgrid, Inc 14 | Vastech SA (PTY) LTD 15 | Walter Schulze 16 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/gogo/protobuf/CONTRIBUTORS: -------------------------------------------------------------------------------- 1 | Anton Povarov 2 | Brian Goff 3 | Clayton Coleman 4 | Denis Smirnov 5 | DongYun Kang 6 | Dwayne Schultz 7 | Georg Apitz 8 | Gustav Paul 9 | Johan Brandhorst 10 | John Shahid 11 | John Tuley 12 | Laurent 13 | Patrick Lee 14 | Peter Edge 15 | Roger Johansson 16 | Sam Nguyen 17 | Sergio Arbeo 18 | Stephen J Day 19 | Tamir Duberstein 20 | Todd Eisenberger 21 | Tormod Erevik Lea 22 | Vyacheslav Kim 23 | Walter Schulze 24 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/gogo/protobuf/gogoproto/gogo.pb.golden: -------------------------------------------------------------------------------- 1 | // Code generated by protoc-gen-go. 2 | // source: gogo.proto 3 | // DO NOT EDIT! 4 | 5 | package gogoproto 6 | 7 | import proto "github.com/gogo/protobuf/proto" 8 | import json "encoding/json" 9 | import math "math" 10 | import google_protobuf "github.com/gogo/protobuf/protoc-gen-gogo/descriptor" 11 | 12 | // Reference proto, json, and math imports to suppress error if they are not otherwise used. 13 | var _ = proto.Marshal 14 | var _ = &json.SyntaxError{} 15 | var _ = math.Inf 16 | 17 | var E_Nullable = &proto.ExtensionDesc{ 18 | ExtendedType: (*google_protobuf.FieldOptions)(nil), 19 | ExtensionType: (*bool)(nil), 20 | Field: 51235, 21 | Name: "gogoproto.nullable", 22 | Tag: "varint,51235,opt,name=nullable", 23 | } 24 | 25 | var E_Embed = &proto.ExtensionDesc{ 26 | ExtendedType: (*google_protobuf.FieldOptions)(nil), 27 | ExtensionType: (*bool)(nil), 28 | Field: 51236, 29 | Name: "gogoproto.embed", 30 | Tag: "varint,51236,opt,name=embed", 31 | } 32 | 33 | var E_Customtype = &proto.ExtensionDesc{ 34 | ExtendedType: (*google_protobuf.FieldOptions)(nil), 35 | ExtensionType: (*string)(nil), 36 | Field: 51237, 37 | Name: "gogoproto.customtype", 38 | Tag: "bytes,51237,opt,name=customtype", 39 | } 40 | 41 | func init() { 42 | proto.RegisterExtension(E_Nullable) 43 | proto.RegisterExtension(E_Embed) 44 | proto.RegisterExtension(E_Customtype) 45 | } 46 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/golang/protobuf/AUTHORS: -------------------------------------------------------------------------------- 1 | # This source code refers to The Go Authors for copyright purposes. 2 | # The master list of authors is in the main Go distribution, 3 | # visible at http://tip.golang.org/AUTHORS. 4 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/golang/protobuf/CONTRIBUTORS: -------------------------------------------------------------------------------- 1 | # This source code was written by the Go contributors. 2 | # The master list of contributors is in the main Go distribution, 3 | # visible at http://tip.golang.org/CONTRIBUTORS. 4 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/pmezard/go-difflib/LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2013, Patrick Mezard 2 | All rights reserved. 3 | 4 | Redistribution and use in source and binary forms, with or without 5 | modification, are permitted provided that the following conditions are 6 | met: 7 | 8 | Redistributions of source code must retain the above copyright 9 | notice, this list of conditions and the following disclaimer. 10 | Redistributions in binary form must reproduce the above copyright 11 | notice, this list of conditions and the following disclaimer in the 12 | documentation and/or other materials provided with the distribution. 13 | The names of its contributors may not be used to endorse or promote 14 | products derived from this software without specific prior written 15 | permission. 16 | 17 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 18 | IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 19 | TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 20 | PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 21 | HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 22 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 23 | TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 24 | PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 25 | LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 26 | NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 27 | SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 28 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2012-2018 Mat Ryer and Tyler Bunnell 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/assert/assertion_format.go.tmpl: -------------------------------------------------------------------------------- 1 | {{.CommentFormat}} 2 | func {{.DocInfo.Name}}f(t TestingT, {{.ParamsFormat}}) bool { 3 | if h, ok := t.(tHelper); ok { h.Helper() } 4 | return {{.DocInfo.Name}}(t, {{.ForwardedParamsFormat}}) 5 | } 6 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/assert/assertion_forward.go.tmpl: -------------------------------------------------------------------------------- 1 | {{.CommentWithoutT "a"}} 2 | func (a *Assertions) {{.DocInfo.Name}}({{.Params}}) bool { 3 | if h, ok := a.t.(tHelper); ok { h.Helper() } 4 | return {{.DocInfo.Name}}(a.t, {{.ForwardedParams}}) 5 | } 6 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/assert/doc.go: -------------------------------------------------------------------------------- 1 | // Package assert provides a set of comprehensive testing tools for use with the normal Go testing system. 2 | // 3 | // Example Usage 4 | // 5 | // The following is a complete example using assert in a standard test function: 6 | // import ( 7 | // "testing" 8 | // "github.com/stretchr/testify/assert" 9 | // ) 10 | // 11 | // func TestSomething(t *testing.T) { 12 | // 13 | // var a string = "Hello" 14 | // var b string = "Hello" 15 | // 16 | // assert.Equal(t, a, b, "The two words should be the same.") 17 | // 18 | // } 19 | // 20 | // if you assert many times, use the format below: 21 | // 22 | // import ( 23 | // "testing" 24 | // "github.com/stretchr/testify/assert" 25 | // ) 26 | // 27 | // func TestSomething(t *testing.T) { 28 | // assert := assert.New(t) 29 | // 30 | // var a string = "Hello" 31 | // var b string = "Hello" 32 | // 33 | // assert.Equal(a, b, "The two words should be the same.") 34 | // } 35 | // 36 | // Assertions 37 | // 38 | // Assertions allow you to easily write test code, and are global funcs in the `assert` package. 39 | // All assertion functions take, as the first argument, the `*testing.T` object provided by the 40 | // testing framework. This allows the assertion funcs to write the failings and other details to 41 | // the correct place. 42 | // 43 | // Every assertion function also takes an optional string message as the final argument, 44 | // allowing custom error messages to be appended to the message the assertion method outputs. 45 | package assert 46 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/assert/errors.go: -------------------------------------------------------------------------------- 1 | package assert 2 | 3 | import ( 4 | "errors" 5 | ) 6 | 7 | // AnError is an error instance useful for testing. If the code does not care 8 | // about error specifics, and only needs to return the error for example, this 9 | // error should be used to make the test code more readable. 10 | var AnError = errors.New("assert.AnError general error for testing") 11 | -------------------------------------------------------------------------------- /src/main/go/vendor/github.com/stretchr/testify/assert/forward_assertions.go: -------------------------------------------------------------------------------- 1 | package assert 2 | 3 | // Assertions provides assertion methods around the 4 | // TestingT interface. 5 | type Assertions struct { 6 | t TestingT 7 | } 8 | 9 | // New makes a new Assertions object for the specified TestingT. 10 | func New(t TestingT) *Assertions { 11 | return &Assertions{ 12 | t: t, 13 | } 14 | } 15 | 16 | //go:generate go run ../_codegen/main.go -output-package=assert -template=assertion_forward.go.tmpl -include-format-funcs 17 | -------------------------------------------------------------------------------- /src/main/go/vendor/gopkg.in/yaml.v2/.travis.yml: -------------------------------------------------------------------------------- 1 | language: go 2 | 3 | go: 4 | - 1.4 5 | - 1.5 6 | - 1.6 7 | - 1.7 8 | - 1.8 9 | - 1.9 10 | - tip 11 | 12 | go_import_path: gopkg.in/yaml.v2 13 | -------------------------------------------------------------------------------- /src/main/go/vendor/gopkg.in/yaml.v2/LICENSE.libyaml: -------------------------------------------------------------------------------- 1 | The following files were ported to Go from C files of libyaml, and thus 2 | are still covered by their original copyright and license: 3 | 4 | apic.go 5 | emitterc.go 6 | parserc.go 7 | readerc.go 8 | scannerc.go 9 | writerc.go 10 | yamlh.go 11 | yamlprivateh.go 12 | 13 | Copyright (c) 2006 Kirill Simonov 14 | 15 | Permission is hereby granted, free of charge, to any person obtaining a copy of 16 | this software and associated documentation files (the "Software"), to deal in 17 | the Software without restriction, including without limitation the rights to 18 | use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies 19 | of the Software, and to permit persons to whom the Software is furnished to do 20 | so, subject to the following conditions: 21 | 22 | The above copyright notice and this permission notice shall be included in all 23 | copies or substantial portions of the Software. 24 | 25 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 26 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 27 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 28 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 29 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 30 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 31 | SOFTWARE. 32 | -------------------------------------------------------------------------------- /src/main/go/vendor/gopkg.in/yaml.v2/NOTICE: -------------------------------------------------------------------------------- 1 | Copyright 2011-2016 Canonical Ltd. 2 | 3 | Licensed under the Apache License, Version 2.0 (the "License"); 4 | you may not use this file except in compliance with the License. 5 | You may obtain a copy of the License at 6 | 7 | http://www.apache.org/licenses/LICENSE-2.0 8 | 9 | Unless required by applicable law or agreed to in writing, software 10 | distributed under the License is distributed on an "AS IS" BASIS, 11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | See the License for the specific language governing permissions and 13 | limitations under the License. 14 | -------------------------------------------------------------------------------- /src/main/go/vendor/gopkg.in/yaml.v2/go.mod: -------------------------------------------------------------------------------- 1 | module "gopkg.in/yaml.v2" 2 | 3 | require ( 4 | "gopkg.in/check.v1" v0.0.0-20161208181325-20d25e280405 5 | ) 6 | -------------------------------------------------------------------------------- /src/main/go/vendor/gopkg.in/yaml.v2/writerc.go: -------------------------------------------------------------------------------- 1 | package yaml 2 | 3 | // Set the writer error and return false. 4 | func yaml_emitter_set_writer_error(emitter *yaml_emitter_t, problem string) bool { 5 | emitter.error = yaml_WRITER_ERROR 6 | emitter.problem = problem 7 | return false 8 | } 9 | 10 | // Flush the output buffer. 11 | func yaml_emitter_flush(emitter *yaml_emitter_t) bool { 12 | if emitter.write_handler == nil { 13 | panic("write handler not set") 14 | } 15 | 16 | // Check if the buffer is empty. 17 | if emitter.buffer_pos == 0 { 18 | return true 19 | } 20 | 21 | if err := emitter.write_handler(emitter, emitter.buffer[:emitter.buffer_pos]); err != nil { 22 | return yaml_emitter_set_writer_error(emitter, "write error: "+err.Error()) 23 | } 24 | emitter.buffer_pos = 0 25 | return true 26 | } 27 | -------------------------------------------------------------------------------- /src/main/go/vendor/modules.txt: -------------------------------------------------------------------------------- 1 | # github.com/davecgh/go-spew v1.1.0 2 | github.com/davecgh/go-spew/spew 3 | # github.com/gogo/protobuf v1.2.1 4 | github.com/gogo/protobuf/gogoproto 5 | github.com/gogo/protobuf/proto 6 | github.com/gogo/protobuf/protoc-gen-gogo/descriptor 7 | # github.com/golang/protobuf v1.3.2 8 | github.com/golang/protobuf/proto 9 | # github.com/pmezard/go-difflib v1.0.0 10 | github.com/pmezard/go-difflib/difflib 11 | # github.com/stretchr/testify v1.4.0 12 | github.com/stretchr/testify/assert 13 | # gopkg.in/yaml.v2 v2.2.2 14 | gopkg.in/yaml.v2 15 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/common/Client.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.common; 2 | 3 | 4 | 5 | import java.net.Socket; 6 | 7 | class Client { 8 | public RequestOrResponse sendReceive(RequestOrResponse requestOrResponse, InetAddressAndPort to) { 9 | try { 10 | var clientSocket = new Socket(to.getAddress(), to.getPort()); 11 | var response = new SocketIO(clientSocket, RequestOrResponse.class).requestResponse(requestOrResponse); 12 | return response; 13 | } catch (Exception e) { 14 | throw new RuntimeException(e); 15 | } 16 | } 17 | } -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/distlog/Journal.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog; 2 | 3 | public class Journal { 4 | } 5 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/distlog/LedgerStorage.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog; 2 | 3 | public class LedgerStorage { 4 | } 5 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/distlog/SortedLedgerStorage.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog; 2 | 3 | import io.netty.buffer.ByteBuf; 4 | 5 | import java.io.IOException; 6 | import java.nio.ByteBuffer; 7 | import java.util.concurrent.ConcurrentSkipListMap; 8 | import java.util.concurrent.atomic.AtomicLong; 9 | 10 | public class SortedLedgerStorage { 11 | AtomicLong size = new AtomicLong(0); 12 | ConcurrentSkipListMap memTable = new ConcurrentSkipListMap(); 13 | private double skipListSizeLimit = 64 * 1024 * 1024L; 14 | 15 | public long addEntry(long ledgerId, long entryId, final ByteBuffer entry) throws IOException { 16 | if (isSizeLimitReached()) { 17 | 18 | } 19 | 20 | EntryKeyValue keyValue = newEntry(ledgerId, entryId, entry); 21 | memTable.putIfAbsent(keyValue, keyValue); 22 | size.addAndGet(keyValue.getLength()); 23 | 24 | // interleavedLedgerStorage.ledgerCache.updateLastAddConfirmed(ledgerId, lac); 25 | return entryId; 26 | } 27 | 28 | boolean isSizeLimitReached() { 29 | return size.get() >= skipListSizeLimit; 30 | } 31 | 32 | private EntryKeyValue newEntry(long ledgerId, long entryId, final ByteBuffer entry) { 33 | byte[] buf; 34 | int offset = 0; 35 | int length = entry.remaining(); 36 | buf = new byte[length]; 37 | entry.get(buf); 38 | return new EntryKeyValue(ledgerId, entryId, buf, offset, length); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/heartbeat/HeartBeatScheduler.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.heartbeat; 2 | 3 | import java.util.concurrent.ScheduledFuture; 4 | import java.util.concurrent.ScheduledThreadPoolExecutor; 5 | import java.util.concurrent.TimeUnit; 6 | 7 | 8 | public class HeartBeatScheduler { 9 | private Runnable action; 10 | private int heartBeatInterval; 11 | 12 | public HeartBeatScheduler(Runnable action, int heartBeatInterval) { 13 | this.action = action; 14 | this.heartBeatInterval = heartBeatInterval; 15 | } 16 | 17 | private ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1); 18 | 19 | private ScheduledFuture scheduledGossipTask = null; 20 | 21 | public void start() { 22 | scheduledGossipTask = executor.scheduleWithFixedDelay(new HeartBeatTask(action), heartBeatInterval, heartBeatInterval, TimeUnit.MILLISECONDS); 23 | } 24 | 25 | private static class HeartBeatTask implements Runnable { 26 | private Runnable action; 27 | 28 | public HeartBeatTask(Runnable action) { 29 | this.action = action; 30 | } 31 | 32 | @Override 33 | public void run() { 34 | action.run(); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/A.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | /** 21 | * Defines internal {@code typedef} for {@link GridArgumentCheck}. Since Java doesn't provide type aliases 22 | * (like Scala, for example) we resort to these types of measures. This is intended for internal 23 | * use only and meant to provide for more terse code when readability of code is not compromised. 24 | */ 25 | @SuppressWarnings({"ExtendsUtilityClass"}) 26 | public class A extends GridArgumentCheck { /* No-op. */ } 27 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/AlwaysFalsePredicate.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | /** 21 | * Defines a predicate which accepts a parameter and always returns {@code false} 22 | * 23 | * @param Type of predicate parameter. 24 | */ 25 | public class AlwaysFalsePredicate implements IgnitePredicate { 26 | /** */ 27 | private static final long serialVersionUID = 0L; 28 | 29 | /** 30 | * Predicate body. 31 | * 32 | * @param e Predicate parameter. 33 | * @return Always false 34 | */ 35 | @Override public boolean apply(E e) { 36 | return false; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/AlwaysTruePredicate.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | /** 21 | * Defines a predicate which accepts a parameter and always returns {@code true} 22 | * 23 | * @param Type of predicate parameter. 24 | */ 25 | public class AlwaysTruePredicate implements IgnitePredicate { 26 | /** */ 27 | private static final long serialVersionUID = 6101914246981105862L; 28 | 29 | /** 30 | * Predicate body. 31 | * 32 | * @param e Predicate parameter. 33 | * @return Always true. 34 | */ 35 | @Override public boolean apply(E e) { 36 | return true; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/F.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | import java.util.UUID; 21 | 22 | /** 23 | * Defines {@code alias} for {@link GridFunc} by extending it. Since Java doesn't provide type aliases 24 | * (like Scala, for example) we resort to these types of measures. This is intended to provide for more 25 | * concise code in cases when readability won't be sacrificed. For more information see {@link GridFunc}. 26 | * @see GridFunc 27 | */ 28 | @SuppressWarnings({"ExtendsUtilityClass"}) 29 | public class F extends GridFunc { 30 | } 31 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/GridSerializableCollection.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | import java.io.Serializable; 21 | import java.util.AbstractCollection; 22 | 23 | /** 24 | * Makes {@link AbstractCollection} as {@link Serializable} and is 25 | * useful for making anonymous serializable collections. It has no 26 | * extra logic or state in addition to {@link AbstractCollection}. 27 | */ 28 | public abstract class GridSerializableCollection extends AbstractCollection implements Serializable { 29 | /** */ 30 | private static final long serialVersionUID = 0L; 31 | 32 | // No-op. 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/GridSerializableIterator.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | import java.io.Serializable; 21 | import java.util.Iterator; 22 | 23 | /** 24 | * Makes {@link Iterator} as {@link Serializable} and is 25 | * useful for making anonymous serializable iterators. 26 | */ 27 | public interface GridSerializableIterator extends Iterator, Serializable { 28 | // No-op. 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/IdentityClosure.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | /** 21 | * Identity closure. 22 | */ 23 | public class IdentityClosure implements IgniteClosure { 24 | /** */ 25 | private static final long serialVersionUID = -6338573080046225172L; 26 | 27 | /** {@inheritDoc} */ 28 | @Override public Object apply(Object o) { 29 | return o; 30 | } 31 | 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/IgnitePredicate.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | import java.io.Serializable; 21 | 22 | /** 23 | * Defines a predicate which accepts a parameter and returns {@code true} or {@code false}. 24 | * 25 | * @param Type of predicate parameter. 26 | */ 27 | public interface IgnitePredicate extends Serializable { 28 | /** 29 | * Predicate body. 30 | * 31 | * @param e Predicate parameter. 32 | * @return Return value. 33 | */ 34 | public boolean apply(E e); 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/IgniteUtils.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.ignite.cluster; 2 | 3 | import java.util.ArrayList; 4 | import java.util.Collection; 5 | import java.util.Iterator; 6 | import java.util.List; 7 | 8 | public class IgniteUtils { 9 | 10 | public static List arrayList(Collection c, IgnitePredicate... p) { 11 | assert c != null; 12 | 13 | return arrayList(c.iterator(), c.size(), p); 14 | } 15 | 16 | /** 17 | * @param c Collection. 18 | * @return Resulting array list. 19 | */ 20 | public static List arrayList(Collection c) { 21 | assert c != null; 22 | 23 | return new ArrayList(c); 24 | } 25 | 26 | /** 27 | * @param c Collection. 28 | * @param cap Initial capacity. 29 | * @param p Optional filters. 30 | * @return Resulting array list. 31 | */ 32 | public static List arrayList(Iterator c, int cap, 33 | IgnitePredicate... p) { 34 | assert c != null; 35 | assert cap >= 0; 36 | 37 | List list = new ArrayList<>(cap); 38 | 39 | while (c.hasNext()) { 40 | T t = c.next(); 41 | 42 | if (F.isAll(t, p)) 43 | list.add(t); 44 | } 45 | 46 | return list; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/P1.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | 21 | /** 22 | * Defines {@code alias} for {@link org.apache.ignite.lang.IgnitePredicate} by extending it. Since Java doesn't provide type aliases 23 | * (like Scala, for example) we resort to these types of measures. This is intended to provide for more 24 | * concise code in cases when readability won't be sacrificed. For more information see {@link org.apache.ignite.lang.IgnitePredicate}. 25 | * @param Type of the free variable, i.e. the element the predicate is called on. 26 | * @see GridFunc 27 | * @see org.apache.ignite.lang.IgnitePredicate 28 | */ 29 | public interface P1 extends IgnitePredicate { /* No-op. */ } 30 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/ServerImpl.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.ignite.cluster; 2 | 3 | 4 | import org.dist.patterns.common.InetAddressAndPort; 5 | import org.dist.patterns.common.RequestOrResponse; 6 | import org.dist.patterns.common.TcpListener; 7 | 8 | import static org.dist.patterns.ignite.cluster.TcpDiscoverySpiState.DISCONNECTED; 9 | 10 | public class ServerImpl { 11 | private InetAddressAndPort listenAddress; 12 | private TcpListener tcpServ; 13 | 14 | public ServerImpl(InetAddressAndPort listenAddress) { 15 | this.listenAddress = listenAddress; 16 | } 17 | 18 | protected TcpDiscoverySpiState spiState = DISCONNECTED; 19 | 20 | public void spiStart(String gridName) { 21 | spiState = DISCONNECTED; 22 | tcpServ = new TcpListener(listenAddress); 23 | } 24 | 25 | public RequestOrResponse process(RequestOrResponse requestOrResponse) { 26 | return requestOrResponse; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/TcpDiscoverySpiState.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | /** 21 | * State of local node {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi}. 22 | */ 23 | public enum TcpDiscoverySpiState { 24 | /** */ 25 | DISCONNECTED, 26 | 27 | /** */ 28 | CONNECTING, 29 | 30 | /** */ 31 | CONNECTED, 32 | 33 | /** */ 34 | DISCONNECTING, 35 | 36 | /** */ 37 | STOPPING, 38 | 39 | /** */ 40 | LEFT, 41 | 42 | /** */ 43 | DUPLICATE_ID, 44 | 45 | /** */ 46 | AUTH_FAILED, 47 | 48 | /** */ 49 | CHECK_FAILED, 50 | 51 | /** */ 52 | LOOPBACK_PROBLEM 53 | } -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/ignite/cluster/U.java: -------------------------------------------------------------------------------- 1 | /* 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 | 18 | package org.dist.patterns.ignite.cluster; 19 | 20 | 21 | /** 22 | * Defines internal {@code typedef} for {@link IgniteUtils}. Since Java doesn't provide type aliases 23 | * (like Scala, for example) we resort to these types of measures. This is intended for internal 24 | * use only and meant to provide for more terse code when readability of code is not compromised. 25 | */ 26 | public class U extends IgniteUtils { /* No-op. */ } 27 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/EntryEventFactory.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public class EntryEventFactory { 4 | public EntryEventImpl create(LocalRegion region, Operation update, K key, V value, Object myId) { 5 | return new EntryEventImpl(region, update, key, value); 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/EntryEventImpl.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public class EntryEventImpl { 4 | private LocalRegion region; 5 | private Operation update; 6 | private Object key; 7 | private Object value; 8 | 9 | public EntryEventImpl(LocalRegion region, Operation update, K key, V value) { 10 | this.region = region; 11 | this.update = update; 12 | this.key = key; 13 | this.value = value; 14 | } 15 | 16 | public LocalRegion getRegion() { 17 | return region; 18 | } 19 | 20 | public Operation getUpdate() { 21 | return update; 22 | } 23 | 24 | public Object getKey() { 25 | return key; 26 | } 27 | 28 | public Object getValue() { 29 | return value; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/InternalDataView.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public interface InternalDataView { 4 | void putEntry(EntryEventImpl entryEvent); 5 | 6 | } 7 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/Region.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public interface Region { 4 | V get(K key); 5 | V put(K key, V value); 6 | } 7 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/TXStateInterface.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public interface TXStateInterface extends InternalDataView { 4 | TransactionId getTransactionId(); 5 | /* 6 | * Only applicable for Distributed transaction. 7 | */ 8 | void precommit(); 9 | 10 | void commit(); 11 | 12 | void rollback(); 13 | 14 | } 15 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/TXStateProxy.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | public interface TXStateProxy extends TXStateInterface { 4 | boolean isInProgress(); 5 | 6 | void lock(); 7 | 8 | void release(); 9 | 10 | void unlock(); 11 | } 12 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/TXStateProxyImpl.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | import java.util.ArrayList; 4 | import java.util.List; 5 | 6 | public class TXStateProxyImpl implements TXStateProxy { 7 | TransactionId id; 8 | private List pendingValue = new ArrayList<>(); 9 | 10 | public TXStateProxyImpl(TransactionId id) { 11 | this.id = id; 12 | } 13 | 14 | @Override 15 | public void putEntry(EntryEventImpl entryEvent) { 16 | pendingValue.add(entryEvent); 17 | } 18 | 19 | @Override 20 | public TransactionId getTransactionId() { 21 | return id; 22 | } 23 | 24 | @Override 25 | public void precommit() { 26 | for (EntryEventImpl entryEvent : pendingValue) { 27 | entryEvent.getRegion().writeLock(); 28 | } 29 | } 30 | 31 | @Override 32 | public void commit() { 33 | for (EntryEventImpl entryEvent : pendingValue) { 34 | entryEvent.getRegion().putInternal(entryEvent.getKey(), entryEvent.getValue()); 35 | } 36 | } 37 | 38 | @Override 39 | public void rollback() { 40 | 41 | } 42 | 43 | @Override 44 | public boolean isInProgress() { 45 | return false; 46 | } 47 | 48 | @Override 49 | public void lock() { 50 | 51 | } 52 | 53 | @Override 54 | public void release() { 55 | 56 | } 57 | 58 | @Override 59 | public void unlock() { 60 | 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/imdg/TransactionId.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.imdg; 2 | 3 | import java.util.UUID; 4 | 5 | public class TransactionId { 6 | private UUID uniqId; 7 | public TransactionId(UUID uniqId) { 8 | this.uniqId = uniqId; 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/session/SessionIdGenerator.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.session; 2 | 3 | import org.apache.zookeeper.common.Time; 4 | import org.apache.zookeeper.server.EphemeralType; 5 | 6 | public class SessionIdGenerator { 7 | 8 | public static long initializeNextSessionId(long id) { 9 | long nextSid; 10 | nextSid = (Time.currentElapsedTime() << 24) >>> 8; 11 | nextSid = nextSid | (id << 56); 12 | if (nextSid == EphemeralType.CONTAINER_EPHEMERAL_OWNER) { 13 | ++nextSid; // this is an unlikely edge case, but check it just in case 14 | } 15 | return nextSid; 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/singularupdatequeue/ExecutorBackedSingularUpdateQueue.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.singularupdatequeue; 2 | 3 | import java.util.concurrent.CompletableFuture; 4 | import java.util.concurrent.ExecutorService; 5 | import java.util.concurrent.Executors; 6 | import java.util.concurrent.Future; 7 | 8 | public class ExecutorBackedSingularUpdateQueue { 9 | private UpdateHandler updateHandler; 10 | private SingularUpdateQueue next; 11 | private ExecutorService singleThreadedExecutor = Executors.newSingleThreadExecutor(); 12 | 13 | public ExecutorBackedSingularUpdateQueue(UpdateHandler updateHandler) { 14 | this.updateHandler = updateHandler; 15 | } 16 | 17 | public ExecutorBackedSingularUpdateQueue(UpdateHandler updateHandler, SingularUpdateQueue next) { 18 | this.updateHandler = updateHandler; 19 | this.next = next; 20 | } 21 | 22 | public void start() { 23 | // 24 | } 25 | 26 | public void submit(P request) { 27 | var completableFuture = new CompletableFuture(); 28 | 29 | Future result = singleThreadedExecutor.submit(() -> { 30 | R update = updateHandler.update(request); 31 | completableFuture.complete(update); 32 | }); 33 | 34 | if (next != null) { 35 | completableFuture.thenAccept(s -> next.submit(s)); 36 | } 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/singularupdatequeue/SynchronizedAccount.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.singularupdatequeue; 2 | 3 | import org.dist.patterns.wal.WriteAheadLog; 4 | 5 | import java.io.*; 6 | 7 | public class SynchronizedAccount { 8 | private int balance; 9 | private WriteAheadLog log; 10 | public SynchronizedAccount(int balance, File dir) { 11 | this.balance = balance; 12 | this.log = WriteAheadLog.openWAL(0, dir); 13 | } 14 | 15 | public synchronized int credit(int amount) { 16 | balance += amount; 17 | writeToFile(balance); 18 | return balance; 19 | } 20 | 21 | private long t = System.nanoTime(); 22 | private volatile long consumeCPU = 0; 23 | 24 | public synchronized int debit(int amount) { 25 | balance -= amount; 26 | writeToFile(balance); 27 | return balance; 28 | } 29 | 30 | private void writeToFile(int balance) { 31 | log.write(balance + ""); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/org/dist/patterns/singularupdatequeue/UpdateHandler.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.singularupdatequeue; 2 | 3 | public interface UpdateHandler { 4 | public V update(U u); 5 | } 6 | -------------------------------------------------------------------------------- /src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=TRACE, stdout 2 | 3 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 4 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 5 | log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n 6 | 7 | # zkclient can be verbose, during debugging it is common to adjust is separately 8 | log4j.logger.org.I0Itec.zkclient.ZkClient=WARN 9 | log4j.logger.org.apache.zookeeper=WARN 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/akkagossip/GossipEnvelope.scala: -------------------------------------------------------------------------------- 1 | package org.dist.akkagossip 2 | 3 | import org.dist.patterns.common.InetAddressAndPort 4 | 5 | case class GossipEnvelope(override val from: InetAddressAndPort, 6 | to: InetAddressAndPort, 7 | latestGossip: Gossip) extends Message(from) { 8 | 9 | } 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/akkagossip/GossipOverview.scala: -------------------------------------------------------------------------------- 1 | package org.dist.akkagossip 2 | 3 | import org.dist.patterns.common.InetAddressAndPort 4 | 5 | import java.nio.charset.StandardCharsets 6 | import java.security.MessageDigest 7 | 8 | case class GossipOverview( 9 | seen: Set[InetAddressAndPort] = Set.empty, 10 | reachability: Reachability = Reachability.empty) { 11 | 12 | lazy val seenDigest: Array[Byte] = { 13 | val bytes = seen.toVector.sorted.map(node => node.getAddress).mkString(",").getBytes(StandardCharsets.UTF_8) 14 | MessageDigest.getInstance("SHA-1").digest(bytes) 15 | } 16 | 17 | override def toString = 18 | s"GossipOverview(reachability = [$reachability], seen = [${seen.mkString(", ")}])" 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/akkagossip/Join.scala: -------------------------------------------------------------------------------- 1 | package org.dist.akkagossip 2 | 3 | import org.dist.patterns.common.InetAddressAndPort 4 | 5 | case class Join(fromAddress: InetAddressAndPort) extends Message(fromAddress) { 6 | 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/akkagossip/Welcome.scala: -------------------------------------------------------------------------------- 1 | package org.dist.akkagossip 2 | 3 | import org.dist.patterns.common.InetAddressAndPort 4 | 5 | case class Welcome(override val from: InetAddressAndPort, joiningWith: InetAddressAndPort, latestGossip: Gossip) extends Message(from) { 6 | 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/BinaryInputArchive.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.io.{DataInputStream, InputStream} 4 | 5 | class BinaryInputArchive(val is: InputStream) { 6 | def readByte() = ds.readByte() 7 | 8 | def readString() = ds.readUTF() 9 | 10 | def readLong(tag: String) = ds.readLong() 11 | 12 | def readInt(tag:String ) = ds.readInt() 13 | 14 | val ds = new DataInputStream(is) 15 | 16 | def readRecord(): QuorumPacket = { 17 | val recordType = ds.readInt() 18 | val zxid = ds.readLong() 19 | val data = readBuffer() 20 | QuorumPacket(recordType, zxid, data) 21 | } 22 | 23 | def readBuffer() = { 24 | val dataSize = ds.readInt() 25 | val data = new Array[Byte](dataSize) 26 | ds.read(data) 27 | data 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/BinaryOutputArchive.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.io.{DataOutputStream, OutputStream} 4 | 5 | class BinaryOutputArchive(val os: OutputStream) { 6 | def writeBuffer(data: Array[Byte], tag: String): Unit = { 7 | ds.writeInt(data.size) 8 | ds.write(data) 9 | } 10 | 11 | def writeString(path: String, tag: String) = { 12 | ds.writeUTF(path) 13 | } 14 | 15 | def writeInt(value: Int, tag: String) = { 16 | ds.writeInt(value) 17 | } 18 | 19 | def writeLong(value:Long, tag: String) = { 20 | ds.writeLong(value) 21 | } 22 | 23 | val ds = new DataOutputStream(os) 24 | 25 | def writeRecord(p: QuorumPacket): Unit = { 26 | ds.writeInt(p.recordType) 27 | ds.writeLong(p.zxid) 28 | writeBuffer(p.data, "data") 29 | ds.flush() 30 | } 31 | 32 | def write(ba:Array[Byte]) = { 33 | ds.write(ba) 34 | } 35 | 36 | def write(byte:Byte, tag:String) = { 37 | ds.write(byte) 38 | } 39 | 40 | def flush() = { 41 | ds.flush() 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/Client.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.net.Socket 4 | import java.util.concurrent.atomic.AtomicInteger 5 | 6 | import org.dist.consensus.zab.api.{ClientRequestOrResponse, SetDataRequest} 7 | import org.dist.kvstore.{InetAddressAndPort, JsonSerDes} 8 | import org.dist.util.SocketIO 9 | 10 | class Client(to:InetAddressAndPort) { 11 | val correlationId = new AtomicInteger(0) 12 | 13 | def setData(path:String, data:String) = { 14 | val clientSocket = new Socket(to.address, to.port) 15 | val request = SetDataRequest(path, data) 16 | val setDataRequest = ClientRequestOrResponse(ClientRequestOrResponse.SetDataKey, JsonSerDes.serialize(request), correlationId.getAndIncrement()) 17 | new SocketIO[ClientRequestOrResponse](clientSocket, classOf[ClientRequestOrResponse]).requestResponse(setDataRequest) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/DataTree.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.util 4 | 5 | class DataTree { 6 | var lastProcessedZxid: Long = 0 7 | val nodes = new util.HashMap[String, Array[Byte]]() 8 | 9 | def processTransaction(txnHeader: TxnHeader, txn: SetDataTxn) = { 10 | val opsCode = txnHeader.opsCode 11 | opsCode match { 12 | case OpsCode.setData ⇒ { 13 | nodes.put(txn.path, txn.data) 14 | } 15 | } 16 | } 17 | 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/ProposeRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | case class ProposeRequest() 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/QuorumPacket.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | case class QuorumPacket(val recordType: Int, var zxid: Long, val data: Array[Byte]) { 4 | def this(recordType:Int, zxid:Long) { 5 | this(recordType, zxid, Array[Byte]()) 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/QuorumPeerConfig.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | 6 | case class QuorumPeerConfig(serverId: Long, electionAddress: InetAddressAndPort, serverAddress: InetAddressAndPort, servers: List[QuorumServer], val dataDir:String) { 7 | val syncLimit = 5 8 | 9 | val tickTime = 100 10 | 11 | val initLimit = 20 12 | } 13 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/QuorumServer.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | case class QuorumServer(id:Int, electionAddress:InetAddressAndPort, serverAddress:InetAddressAndPort) { 6 | def this(id:Int, hostName:String, electionPort:Int, serverPort:Int) { 7 | this(id, InetAddressAndPort.create(hostName, electionPort), InetAddressAndPort.create(hostName, serverPort)) 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/ServerCnxn.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.net.{InetSocketAddress, ServerSocket, Socket} 4 | 5 | import org.dist.consensus.zab.api.ClientRequestOrResponse 6 | import org.dist.kvstore.InetAddressAndPort 7 | import org.dist.queue.common.Logging 8 | import org.dist.queue.utils.Utils 9 | import org.dist.util.SocketIO 10 | 11 | //handles client communication with zk 12 | class ServerCnxn(serverAddress:InetAddressAndPort, val zk:LeaderZookeeperServer) extends Thread with Logging { 13 | var serverSocket: ServerSocket = null 14 | 15 | override def run() = { 16 | listen() 17 | } 18 | 19 | def listen(): Unit = { 20 | Utils.swallow({ 21 | serverSocket = new ServerSocket() 22 | serverSocket.bind(new InetSocketAddress(serverAddress.address, serverAddress.port)) 23 | info(s"Listening on ${serverAddress}") 24 | while (true) { 25 | val socket: Socket = serverSocket.accept() 26 | new SocketIO(socket, classOf[ClientRequestOrResponse]).readHandleWithSocket((request, clientSocket) ⇒ { 27 | zk.submitRequest(request, clientSocket) 28 | }) 29 | } 30 | } 31 | ) 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/SynRequestProcessor.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.io.{File, FileOutputStream} 4 | import java.util.Random 5 | 6 | import org.dist.queue.common.Logging 7 | 8 | class SynRequestProcessor(zks: ZookeeperServer, 9 | nextProcessor: RequestProcessor) extends RequestProcessor with Logging { 10 | private val r = new Random(System.nanoTime) 11 | 12 | private var logCount = 0 13 | var logStream:FileOutputStream = null 14 | var logArchive:BinaryOutputArchive = null 15 | 16 | def run() = { 17 | } 18 | 19 | override def processRequest(request: Request): Unit = { 20 | val lastZxidSeen:Long = -1 21 | val hdr: TxnHeader = request.txnHeader 22 | if (hdr.zxid < lastZxidSeen) warn("Current zxid " + hdr.zxid + " is <= " + lastZxidSeen) 23 | val txn: SetDataTxn = request.txn 24 | if (logStream == null) { 25 | logStream = new FileOutputStream(new File(zks.dataLogDir(), zks.getLogName(hdr.zxid))) 26 | logArchive = new BinaryOutputArchive(logStream) 27 | } 28 | 29 | //serialize hdr 30 | //serialize txn 31 | logArchive.writeBuffer(request.serializeTxn(), "TxnEntry") 32 | logArchive.write(0x42.toByte, "EOR") 33 | 34 | //TODO take snapshot after specified no. of log entries 35 | //TODO: Batch sync instead of syncing each time. 36 | logStream.getFD.sync() 37 | logCount += 1 38 | nextProcessor.processRequest(request) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/Vote.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | case class Vote(id: Long, zxid: Long) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/consensus/zab/api/ClientRequestOrResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab.api 2 | 3 | object ClientRequestOrResponse { 4 | val SetDataKey: Short = 0 5 | val GetDataKey: Short = 1 6 | } 7 | 8 | case class SetDataRequest(path:String, data:String) 9 | case class GetDataRequest(path:String) 10 | case class SetDataResponse(path:String) 11 | case class GetDataResponse(data:String) 12 | 13 | case class ClientRequestOrResponse(val requestId: Short, val messageBodyJson: String, val correlationId: Int) 14 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/ApplicationState.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.dist.kvstore; 19 | 20 | public enum ApplicationState 21 | { 22 | LOAD, 23 | SCHEMA, 24 | DC, 25 | RACK, 26 | RELEASE_VERSION, 27 | REMOVAL_COORDINATOR, 28 | @Deprecated INTERNAL_IP, //Deprecated and unused in 4.0, stop publishing in 5.0, reclaim in 6.0 29 | @Deprecated RPC_ADDRESS, // ^ Same 30 | HOST_ID, 31 | TOKENS 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/DatabaseConfiguration.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util 4 | import scala.jdk.CollectionConverters._ 5 | 6 | case class DatabaseConfiguration(seeds:Set[InetAddressAndPort], systemDir:String = System.getProperty(("java.io.tmpdir"))) { 7 | def getSystemDir(): String = systemDir 8 | 9 | def getClusterName() = "TestCluster" 10 | 11 | 12 | def nonLocalSeeds(localEndpoint:InetAddressAndPort) = { 13 | seeds.filter(address => address != localEndpoint).toList.asJava 14 | } 15 | 16 | } 17 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/FBUtilities.java: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore; 2 | 3 | import org.dist.util.Networks; 4 | 5 | import java.math.BigInteger; 6 | import java.security.MessageDigest; 7 | 8 | public class FBUtilities { 9 | 10 | public static BigInteger hash(String data) 11 | { 12 | byte[] result = hash("MD5", data.getBytes()); 13 | BigInteger hash = new BigInteger(result); 14 | return hash.abs(); 15 | } 16 | 17 | 18 | public static byte[] hash(String type, byte[] data) 19 | { 20 | byte[] result = null; 21 | try 22 | { 23 | MessageDigest messageDigest = MessageDigest.getInstance(type); 24 | result = messageDigest.digest(data); 25 | } 26 | catch (Exception e) 27 | { 28 | throw new RuntimeException(e); 29 | } 30 | return result; 31 | } 32 | 33 | public static InetAddressAndPort getBroadcastAddressAndPort() { 34 | return new InetAddressAndPort(new Networks().ipv4Address(), 8000); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/FailureDetector.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util 4 | 5 | import org.dist.queue.common.Logging 6 | 7 | object FailureDetector { 8 | def isAlive(sp:InetAddressAndPort) = true 9 | } 10 | 11 | class FailureDetector(gossiper: Gossiper) extends Logging { 12 | private val arrivalSamples_ = new util.Hashtable[InetAddressAndPort, ArrivalWindow] 13 | private val sampleSize_ = 1000 14 | private val phiSuspectThreshold_ = 5 15 | private val phiConvictThreshold_ = 8 16 | /* The Failure Detector has to have been up for atleast 1 min. */ 17 | private val uptimeThreshold_ = 60000 18 | 19 | def report(ep: InetAddressAndPort): Unit = { 20 | val now = System.currentTimeMillis 21 | var hbWnd = arrivalSamples_.get(ep) 22 | if (hbWnd == null) { 23 | hbWnd = new ArrivalWindow(sampleSize_) 24 | arrivalSamples_.put(ep, hbWnd) 25 | } 26 | hbWnd.add(now) 27 | } 28 | 29 | def intepret(ep: InetAddressAndPort): Unit = { 30 | val hbWnd = arrivalSamples_.get(ep) 31 | if (hbWnd == null) return 32 | val now = System.currentTimeMillis 33 | /* We need this so that we do not suspect a convict. */ val isConvicted = false 34 | val phi = hbWnd.phi(now) 35 | info("PHI for " + ep + " : " + phi) 36 | if (!isConvicted && phi > phiSuspectThreshold_) { 37 | gossiper.suspect(ep) 38 | } 39 | } 40 | } 41 | 42 | 43 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/GossipDigestAck.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util 4 | 5 | import scala.jdk.CollectionConverters._ 6 | 7 | object GossipDigestAck { 8 | def create(gDigestList: List[GossipDigest], epStateMap: Map[InetAddressAndPort, EndPointState]) = { 9 | val map: util.Map[String, EndPointState] = new util.HashMap[String, EndPointState]() 10 | val set = epStateMap.keySet 11 | for (key <- set) { 12 | val newKey = s"${key.address.getHostAddress}:${key.port}" 13 | map.put(newKey, epStateMap.asJava.get(key)) 14 | } 15 | GossipDigestAck(gDigestList, map.asScala.toMap) 16 | } 17 | } 18 | 19 | case class GossipDigestAck(val gDigestList: List[GossipDigest], 20 | val epStateMap: Map[String, EndPointState]) { 21 | 22 | def digestList = if (gDigestList == null) List[GossipDigest]() else gDigestList 23 | 24 | def stateMap() = { 25 | if (epStateMap == null) { 26 | new util.HashMap[InetAddressAndPort, EndPointState]().asScala 27 | } else { 28 | val map: util.Map[InetAddressAndPort, EndPointState] = new util.HashMap[InetAddressAndPort, EndPointState]() 29 | val set = epStateMap.keySet 30 | for (key <- set) { 31 | val splits = key.split(":") 32 | map.put(InetAddressAndPort.create(splits(0), splits(1).toInt), epStateMap.asJava.get(key)) 33 | } 34 | map.asScala 35 | } 36 | } 37 | 38 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/GossipDigestAck2.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util 4 | 5 | import scala.jdk.CollectionConverters._ 6 | 7 | object GossipDigestAck2 { 8 | def create(epStateMap: Map[InetAddressAndPort, EndPointState]):GossipDigestAck2 = { 9 | val map: util.Map[String, EndPointState] = new util.HashMap[String, EndPointState]() 10 | val set = epStateMap.keySet 11 | for (key <- set) { 12 | val newKey = s"${key.address.getHostAddress}:${key.port}" 13 | map.put(newKey, epStateMap.asJava.get(key)) 14 | } 15 | GossipDigestAck2(map) 16 | } 17 | } 18 | case class GossipDigestAck2(val epStateMap: util.Map[String, EndPointState]) { 19 | def stateMap(): util.Map[InetAddressAndPort, EndPointState] = { 20 | if (epStateMap == null) { 21 | new util.HashMap[InetAddressAndPort, EndPointState]() 22 | } else { 23 | val map: util.Map[InetAddressAndPort, EndPointState] = new util.HashMap[InetAddressAndPort, EndPointState]() 24 | val set = epStateMap.keySet.asScala 25 | for (key <- set) { 26 | val splits = key.split(":") 27 | map.put(InetAddressAndPort.create(splits(0), splits(1).toInt), epStateMap.get(key)) 28 | } 29 | map 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/GossipDigestSyn.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util 4 | 5 | case class GossipDigestSyn(clusterName: String, gDigests: util.List[GossipDigest]) -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/Header.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | case class Header(val from: InetAddressAndPort, messageType: Stage, verb: Verb, val id:String = Message.nextId) -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/HeartBeatState.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util.concurrent.atomic.AtomicInteger 4 | 5 | case class HeartBeatState(generation:Int, version:Int, heartBeat:AtomicInteger = new AtomicInteger(0)) { 6 | def updateVersion(version:Int) = { 7 | heartBeat.incrementAndGet() 8 | HeartBeatState(generation, version, heartBeat) 9 | } 10 | 11 | def updateHeartBeat(version:Int) = { 12 | heartBeat.incrementAndGet() 13 | HeartBeatState(generation, version, heartBeat) 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/IPartitioner.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.dist.kvstore; 20 | 21 | import java.math.BigInteger; 22 | import java.util.Comparator; 23 | 24 | public interface IPartitioner 25 | { 26 | public BigInteger hash(String key); 27 | 28 | /** transform key to on-disk format s.t. keys are stored in node comparison order. 29 | * this lets bootstrap rip out parts of the sstable sequentially instead of doing random seeks. */ 30 | public String decorateKey(String key); 31 | 32 | public String undecorateKey(String decoratedKey); 33 | 34 | public Comparator getReverseDecoratedKeyComparator(); 35 | } 36 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/InetAddressAndPort.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.net.InetAddress 4 | import java.util.Comparator 5 | 6 | object InetAddressAndPort { 7 | //FIXME: Remove this. 8 | def create(hostIp:String, port:Int) = { 9 | new InetAddressAndPort(InetAddress.getByName(hostIp), port) 10 | } 11 | } 12 | 13 | case class InetAddressAndPort(address: InetAddress, port: Int) { 14 | var defaultPort: Int = 7000 15 | override def toString = "[%s,%d]".format(address.getHostAddress, port) 16 | } 17 | 18 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/Memtable.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | class Memtable(table:String, columnFamily:String) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/Message.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util.concurrent.atomic.AtomicInteger 4 | 5 | object Message { 6 | private val messageId = new AtomicInteger(0) 7 | 8 | def nextId = { 9 | GuidGenerator.guid 10 | } 11 | } 12 | 13 | case class Message(header:Header, payloadJson:String) 14 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/RowMutation.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | case class RowMutation(table:String, key:String, value:String) 4 | 5 | case class RowMutationMessage(correlationId:Int, rowMutation:RowMutation) 6 | 7 | case class QurorumResponse(messages:List[Message]) 8 | 9 | case class RowMutationResponse(correlationId:Int, key:String, success:Boolean) 10 | 11 | case class QuorumResponse(values:List[RowMutationResponse]) -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/StorageMetadata.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | case class StorageMetadata(token:String, generation:Int) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/TokenMetadata.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.math.BigInteger 4 | import java.util 5 | import java.util.concurrent.locks.{ReadWriteLock, ReentrantReadWriteLock} 6 | 7 | class TokenMetadata { 8 | def getToken(ep: InetAddressAndPort) = { 9 | InetAddressAndPortToTokenMap.get(ep) 10 | } 11 | 12 | /* Use this lock for manipulating the token map */ 13 | private val lock = new ReentrantReadWriteLock(true) 14 | private var tokenToInetAddressAndPortMap = new util.HashMap[BigInteger, InetAddressAndPort]() 15 | private var InetAddressAndPortToTokenMap = new util.HashMap[InetAddressAndPort, BigInteger] 16 | 17 | def update(token: BigInteger, endpoint: InetAddressAndPort) = { 18 | lock.writeLock.lock() 19 | try { 20 | val oldToken = InetAddressAndPortToTokenMap.get(endpoint) 21 | if (oldToken != null) tokenToInetAddressAndPortMap.remove(oldToken) 22 | println(s"Updating token ${token} for endpoint ${endpoint}") 23 | tokenToInetAddressAndPortMap.put(token, endpoint) 24 | InetAddressAndPortToTokenMap.put(endpoint, token) 25 | 26 | println(s"Token map is now ${InetAddressAndPortToTokenMap}") 27 | 28 | } finally lock.writeLock.unlock() 29 | } 30 | 31 | def cloneTokenEndPointMap = { 32 | tokenToInetAddressAndPortMap.clone().asInstanceOf[util.HashMap[BigInteger, InetAddressAndPort]] 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/Verb.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.dist.kvstore; 19 | 20 | public enum Verb 21 | { 22 | GOSSIP_DIGEST_SYN, 23 | GOSSIP_DIGEST_ACK, 24 | GOSSIP_DIGEST_ACK2, 25 | RESPONSE, 26 | ROW_MUTATION, 27 | GET_CF 28 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/VersionGenerator.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util.concurrent.atomic.AtomicInteger 4 | 5 | //needs to be singleton per server instance along with Gossiper 6 | class VersionGenerator { 7 | private val version = new AtomicInteger(0) 8 | def incrementAndGetVersion: Int = version.incrementAndGet 9 | 10 | def currentVersion = version.get() 11 | 12 | def reset() = version.set(0) 13 | } 14 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/VersionedValue.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | case class VersionedValue(value:String, version:Int) {} 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/client/Client.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.client 2 | 3 | import org.dist.kvstore._ 4 | import org.dist.util.Networks 5 | 6 | class Client(bootstrapServer: InetAddressAndPort) { 7 | private val socketClient = new SocketClient 8 | def put(table: String, key: String, value: String) = { 9 | val mutation = RowMutation(table, key, value) 10 | 11 | val header = Header(InetAddressAndPort(new Networks().ipv4Address, 8000) 12 | , Stage.MUTATION, Verb.ROW_MUTATION) 13 | val message = Message(header, JsonSerDes.serialize(mutation)) 14 | val responseMessage: Message = socketClient.sendReceiveTcp(message, bootstrapServer) 15 | val responses: QuorumResponse = JsonSerDes.deserialize(responseMessage.payloadJson.getBytes, classOf[QuorumResponse]) 16 | responses.values 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/client/SocketClient.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.client 2 | 3 | import java.net.Socket 4 | 5 | import org.dist.kvstore.{InetAddressAndPort, Message} 6 | import org.dist.util.SocketIO 7 | 8 | class SocketClient { 9 | def sendReceiveTcp(message: Message, to: InetAddressAndPort) = { 10 | val clientSocket = new Socket(to.address, to.port) 11 | val responseMessage = new SocketIO[Message](clientSocket, classOf[Message]).requestResponse(message) 12 | responseMessage 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/testapp/Node1.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.testapp 2 | 3 | import org.dist.kvstore.{DatabaseConfiguration, InetAddressAndPort, StorageService} 4 | import org.dist.util.Networks 5 | import Utils._ 6 | import org.dist.kvstore.testapp.GossipTestApp.localIpAddress 7 | 8 | object Node1 extends App { 9 | val localIpAddress = new Networks().ipv4Address 10 | private val node1Endpoint = InetAddressAndPort(localIpAddress, 8000) 11 | private val node1ClientEndpoint = InetAddressAndPort(localIpAddress, 9000) 12 | 13 | val seedConfig = DatabaseConfiguration(Set(node1Endpoint), createDbDir("node1")) 14 | 15 | val node1 = new StorageService(node1ClientEndpoint, node1Endpoint, seedConfig) 16 | 17 | node1.start() 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/testapp/Node2.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.testapp 2 | 3 | import org.dist.kvstore.testapp.GossipTestApp.localIpAddress 4 | import org.dist.kvstore.testapp.Utils.createDbDir 5 | import org.dist.kvstore.{DatabaseConfiguration, InetAddressAndPort, StorageService} 6 | import org.dist.util.Networks 7 | 8 | object Node2 extends App { 9 | 10 | val localIpAddress = new Networks().ipv4Address 11 | 12 | private val node1Endpoint = InetAddressAndPort(localIpAddress, 8000) 13 | val seedConfig = DatabaseConfiguration(Set(node1Endpoint), createDbDir("node2")) 14 | 15 | private val node2Endpoint = InetAddressAndPort(localIpAddress, 8001) 16 | private val node2ClientEndpoint = InetAddressAndPort(localIpAddress, 9001) 17 | 18 | val node2 = new StorageService(node2ClientEndpoint, node2Endpoint, seedConfig) 19 | 20 | node2.start() 21 | 22 | 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/testapp/Node3.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.testapp 2 | 3 | import org.dist.kvstore.testapp.Utils.createDbDir 4 | import org.dist.kvstore.{DatabaseConfiguration, InetAddressAndPort, StorageService} 5 | import org.dist.util.Networks 6 | 7 | object Node3 extends App { 8 | val localIpAddress = new Networks().ipv4Address 9 | private val node1Endpoint = InetAddressAndPort(localIpAddress, 8000) 10 | val seedConfig = DatabaseConfiguration(Set(node1Endpoint), createDbDir("node3")) 11 | 12 | private val node3Endpoint = InetAddressAndPort(localIpAddress, 8002) 13 | private val node3ClientEndpoint = InetAddressAndPort(localIpAddress, 9003) 14 | val node3 = new StorageService(node3ClientEndpoint, node3Endpoint, seedConfig) 15 | 16 | node3.start() 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/kvstore/testapp/Utils.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore.testapp 2 | 3 | import java.io.File 4 | import java.nio.file.{Path, Paths} 5 | 6 | object Utils { 7 | 8 | def createDbDir(dbName: String) = { 9 | val tmpDir: Path = Paths.get(System.getProperty("java.io.tmpdir")) 10 | val node1Dir: String = tmpDir.toString + File.separator + dbName 11 | new File(node1Dir).mkdirs() 12 | node1Dir 13 | } 14 | } 15 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/partitioning/KeyRangePartitioning.scala: -------------------------------------------------------------------------------- 1 | package org.dist.partitioning 2 | 3 | import java.util 4 | 5 | class Storage { 6 | val map = new util.HashMap[String, String]() 7 | } 8 | 9 | class KeyRangePartitioning { 10 | val storages = List(new Storage, new Storage, new Storage) 11 | 12 | def put(key:String, value:String): Unit = { 13 | val index = partitionFor(key) 14 | val storage = storages(index) 15 | storage.map.put(key, value) 16 | } 17 | 18 | def partitionFor(key:String) = { 19 | 0 20 | } 21 | 22 | 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/replicatedlog/ReplicatedKVStore.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog 2 | 3 | import java.io.File 4 | 5 | import org.dist.patterns.wal.Wal 6 | 7 | class ReplicatedKVStore(walDir:File) { 8 | val replicatedWal = new ReplicatedWal(walDir) 9 | 10 | } 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/replicatedlog/ReplicatedWal.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog 2 | 3 | import java.io.File 4 | 5 | import org.dist.patterns.wal.{Wal, WalEntry} 6 | 7 | class ReplicatedWal(walDir:File) { 8 | def entries(from: Long, to: Long) = { 9 | wal.entries(from, to) 10 | } 11 | 12 | val wal = Wal.create(walDir) 13 | 14 | var highWaterMark = 0L 15 | 16 | def readAll() = wal.readAll() 17 | 18 | def lastLogEntryId = wal.lastLogEntryId 19 | 20 | def isUptoDate(entryId:Long) = { 21 | wal.lastLogEntryId == entryId 22 | } 23 | 24 | def truncate(entryId:Long) = { 25 | wal.truncate(entryId) 26 | } 27 | 28 | def append(bytes: Array[Byte]):Long = { 29 | wal.writeEntry(bytes) 30 | } 31 | 32 | def updateHighWaterMark(entryId:Long) = { 33 | highWaterMark = entryId 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/replicatedlog/api/VoteRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog.api 2 | 3 | object RequestKeys { 4 | val RequestVoteKey: Short = 0 5 | val AppendEntriesKey: Short = 1 6 | } 7 | 8 | case class VoteRequest(serverId:Long, lastXid:Long) 9 | 10 | case class VoteResponse(serverId:Long, lastXid:Long) -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/replicatedlog/heartbeat/Peer.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog.heartbeat 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | import org.dist.patterns.replicatedlog.Leader 5 | import org.dist.rapid.SocketClient 6 | 7 | 8 | case class Peer(id:Int, address:InetAddressAndPort) 9 | 10 | case class PeerProxy(peerInfo: Peer, client: SocketClient = null, var matchIndex: Long = 0, heartbeatSender: PeerProxy ⇒ Unit) { 11 | 12 | def heartbeatSenderWrapper() = { 13 | heartbeatSender(this) 14 | } 15 | 16 | val heartBeat = new HeartBeatScheduler(heartbeatSenderWrapper) 17 | 18 | def start(): Unit = { 19 | heartBeat.start() 20 | } 21 | 22 | def stop() = { 23 | heartBeat.cancel() 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/wal/KVStore.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.wal 2 | 3 | import java.io.{ByteArrayInputStream, File} 4 | 5 | import scala.collection.mutable 6 | import scala.collection.mutable.ListBuffer 7 | 8 | class KVStore(walDir:File) { 9 | val kv = new mutable.HashMap[String, String]() 10 | val wal = Wal.create(walDir) 11 | 12 | 13 | def put(key:String, value:String): Unit = { 14 | wal.writeEntry(SetValueCommand(key, value).serialize()) 15 | 16 | kv.put(key, value) 17 | } 18 | 19 | def get(key: String): Option[String] = kv.get(key) 20 | 21 | def close = { 22 | wal.close() 23 | kv.clear() 24 | } 25 | 26 | def applyLog() = { 27 | val entries: ListBuffer[WalEntry] = wal.readAll() 28 | entries.foreach(entry ⇒ { 29 | val command = SetValueCommand.deserialize(new ByteArrayInputStream(entry.data)) 30 | kv.put(command.key, command.value 31 | ) 32 | }) 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/patterns/wal/SetValueCommand.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.wal 2 | 3 | import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream, InputStream} 4 | 5 | object SetValueCommand { 6 | def deserialize(is:InputStream) = { 7 | val daos = new DataInputStream(is) 8 | val key = daos.readUTF() 9 | val value = daos.readUTF() 10 | SetValueCommand(key, value) 11 | } 12 | } 13 | 14 | case class SetValueCommand(val key:String, val value:String) { 15 | def serialize() = { 16 | val baos = new ByteArrayOutputStream 17 | val dataStream = new DataOutputStream(baos) 18 | dataStream.writeUTF(key) 19 | dataStream.writeUTF(value) 20 | baos.toByteArray 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/admin/CreateTopicCommand.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.admin 2 | 3 | import org.I0Itec.zkclient.ZkClient 4 | import org.dist.queue.utils.{AdminUtils, ZkUtils} 5 | 6 | object CreateTopicCommand { 7 | def createTopic(zkClient:ZkClient, topicName:String, numPartitions:Int = 1, replicationFactor: Int = 1, replicaAssignmentStr: String = ""): Unit = { 8 | val brokerList = ZkUtils.getSortedBrokerList(zkClient) 9 | val partitionReplicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor) 10 | AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(topicName, partitionReplicaAssignment, zkClient) 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/FindCoordinatorRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | object CoordinatorType { 4 | val GROUP = 0 5 | val TRANSACTION = 1 6 | } 7 | 8 | //KeyType is CoordinatorType. Keeping it that way to be closer to how its in Kafka codebase 9 | case class FindCoordinatorRequest(groupId:String, keyType:Int) 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/FindCoordinatorResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | case class FindCoordinatorResponse(host:String, port:Int) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/LeaderAndIsrResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | case class LeaderAndIsrResponse(val correlationId: Int, 4 | responseMap: Map[(String, Int), Short], 5 | errorCode: Short = 0) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/ProducerResponseStatus.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | import org.dist.queue.common.TopicAndPartition 4 | 5 | case class ProducerResponse(val correlationId: Int, 6 | status: Map[TopicAndPartition, ProducerResponseStatus]) 7 | 8 | case class ProducerResponseStatus(error: Short, offset: Long) 9 | 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/RequestOrResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | import org.dist.kvstore.JsonSerDes 4 | 5 | object Request { 6 | val FollowerReplicaFetcherId: Int = 0 7 | val OrdinaryConsumerId: Int = -1 8 | val DebuggingConsumerId: Int = -2 9 | } 10 | 11 | 12 | object RequestKeys { 13 | val ProduceKey: Short = 0 14 | val FetchKey: Short = 1 15 | val OffsetsKey: Short = 2 16 | val GetMetadataKey: Short = 3 17 | val LeaderAndIsrKey: Short = 4 18 | val StopReplicaKey: Short = 5 19 | val UpdateMetadataKey: Short = 6 20 | val ControlledShutdownKey: Short = 7 21 | val FindCoordinatorKey:Short = 8 22 | } 23 | 24 | case class RequestOrResponse(val requestId: Short, val messageBodyJson: String, val correlationId: Int) { 25 | def serialize(): String = { 26 | JsonSerDes.serialize(this) 27 | } 28 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/TopicMetadataRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | case class TopicMetadataRequest(val versionId: Short, 4 | val correlationId: Int, 5 | val clientId: String, 6 | val topics: Seq[String]) 7 | 8 | object TopicMetadataRequest { 9 | val CurrentVersion = 0.shortValue 10 | val DefaultClientId = "" 11 | 12 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/TopicMetadataResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | import org.dist.queue.common.ErrorMapping 4 | import org.dist.queue.utils.ZkUtils.Broker 5 | 6 | case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata], 7 | val correlationId: Int) 8 | 9 | case class TopicMetadata(topic: String, partitionsMetadata: Seq[PartitionMetadata], errorCode: Short = ErrorMapping.NoError) 10 | 11 | case class PartitionMetadata(partitionId: Int, 12 | val leader: Option[Broker], 13 | replicas: Seq[Broker], 14 | isr: Seq[Broker] = Seq.empty, 15 | errorCode: Short = ErrorMapping.NoError) 16 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/api/UpdateMetadataResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | case class UpdateMetadataResponse(val correlationId: Int, 4 | errorCode: Short = 0) -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/client/common/ClientUtils.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.common 2 | 3 | import org.dist.kvstore.{InetAddressAndPort, JsonSerDes} 4 | import org.dist.queue.api._ 5 | import org.dist.queue.network.SocketClient 6 | 7 | class ClientUtils { 8 | val socketClient = new SocketClient 9 | 10 | 11 | def fetchTopicMetadata(topics: Set[String], correlationId: Int, clientId:String, bootstrapBroker:InetAddressAndPort): Seq[TopicMetadata] = { 12 | val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationId, clientId, topics.toSeq) 13 | val response = socketClient.sendReceiveTcp(new RequestOrResponse(RequestKeys.GetMetadataKey, JsonSerDes.serialize(topicMetadataRequest), correlationId), bootstrapBroker) 14 | val topicMetadataResponse = JsonSerDes.deserialize(response.messageBodyJson.getBytes(), classOf[TopicMetadataResponse]) 15 | topicMetadataResponse.topicsMetadata 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/client/common/PartitionAndLeader.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.common 2 | 3 | case class PartitionAndLeader(topic: String, partitionId: Int, leaderBrokerIdOpt: Option[Int]) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/client/consumer/ConsumerConfig.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.consumer 2 | 3 | import org.dist.queue.api.OffsetRequest 4 | 5 | object ConsumerConfig { 6 | val DefaultClientId: String = "" 7 | 8 | val RefreshMetadataBackoffMs = 200 9 | val SocketTimeout = 30 * 1000 10 | val SocketBufferSize = 64*1024 11 | val FetchSize = 1024 * 1024 12 | val MaxFetchSize = 10*FetchSize 13 | val DefaultFetcherBackoffMs = 1000 14 | val AutoCommit = true 15 | val AutoCommitInterval = 60 * 1000 16 | val MaxQueuedChunks = 2 17 | val MaxRebalanceRetries = 4 18 | val AutoOffsetReset = OffsetRequest.LargestTimeString 19 | val ConsumerTimeoutMs = -1 20 | val MinFetchBytes = 1 21 | val MaxFetchWaitMs = 100 22 | val MirrorTopicsWhitelist = "" 23 | val MirrorTopicsBlacklist = "" 24 | } 25 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/client/producer/DefaultPartitioner.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.producer 2 | 3 | import org.dist.queue.utils.Utils 4 | 5 | class DefaultPartitioner[T] extends Partitioner[T] { 6 | private val random = new java.util.Random 7 | 8 | def partition(key: T, numPartitions: Int): Int = { 9 | Utils.abs(key.hashCode) % numPartitions 10 | } 11 | } 12 | 13 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/client/producer/Practitioner.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.producer 2 | 3 | /** 4 | * A partitioner controls the mapping between user-provided keys and kafka partitions. Users can implement a custom 5 | * partitioner to change this mapping. 6 | * 7 | * Implementations will be constructed via reflection and are required to have a constructor that takes a single 8 | * VerifiableProperties instance--this allows passing configuration properties into the partitioner implementation. 9 | */ 10 | trait Partitioner[T] { 11 | /** 12 | * Uses the key to calculate a partition bucket id for routing 13 | * the data to the appropriate broker partition 14 | * @return an integer between 0 and numPartitions-1 15 | */ 16 | def partition(key: T, numPartitions: Int): Int 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/ErrorMapping.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | object ErrorMapping { 4 | def codeFor(value: Class[Throwable]): Short = UnknownCode 5 | 6 | val UnknownCode : Short = -1 7 | val NoError : Short = 0 8 | val OffsetOutOfRangeCode : Short = 1 9 | val InvalidMessageCode : Short = 2 10 | val UnknownTopicOrPartitionCode : Short = 3 11 | val InvalidFetchSizeCode : Short = 4 12 | val LeaderNotAvailableCode : Short = 5 13 | val NotLeaderForPartitionCode : Short = 6 14 | val RequestTimedOutCode: Short = 7 15 | val BrokerNotAvailableCode: Short = 8 16 | val ReplicaNotAvailableCode: Short = 9 17 | val MessageSizeTooLargeCode: Short = 10 18 | val StaleControllerEpochCode: Short = 11 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/InvalidMessageSizeException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class InvalidMessageSizeException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/KafkaException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | /** 4 | * Generic Kafka exception 5 | */ 6 | class KafkaException(message: String, t: Throwable) extends RuntimeException(message, t) { 7 | def this(message: String) = this(message, null) 8 | def this(t: Throwable) = this("", t) 9 | } 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/KafkaStorageException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | import java.io.IOException 4 | 5 | case class KafkaStorageException(str: String, e: IOException) extends RuntimeException(str, e) { 6 | 7 | def this(str:String) = { 8 | this(str, null) 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/KafkaZookeeperClient.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | import java.util.concurrent.atomic.AtomicReference 4 | 5 | import org.I0Itec.zkclient.ZkClient 6 | import org.dist.queue.server.Config 7 | import org.dist.queue.utils.ZKStringSerializer 8 | 9 | object KafkaZookeeperClient { 10 | private val INSTANCE = new AtomicReference[ZkClient](null) 11 | 12 | def getZookeeperClient(config: Config, connect: Config => String): ZkClient = { 13 | // TODO: This cannot be a singleton since unit tests break if we do that 14 | // INSTANCE.compareAndSet(null, new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, 15 | // ZKStringSerializer)) 16 | INSTANCE.set(new ZkClient(connect(config), config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, 17 | ZKStringSerializer)) 18 | INSTANCE.get() 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/LeaderNotAvailableException.scala: -------------------------------------------------------------------------------- 1 | /** 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 | 18 | package org.dist.queue.common 19 | 20 | /** 21 | * Thrown when a request is made for partition, but no leader exists for that partition 22 | */ 23 | class LeaderNotAvailableException(message: String, cause: Throwable) extends RuntimeException(message, cause) { 24 | def this(message: String) = this(message, null) 25 | def this() = this(null, null) 26 | } 27 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/NoBrokersForPartitionException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class NoBrokersForPartitionException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/NotLeaderForPartitionException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class NotLeaderForPartitionException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/ReplicaNotAvailableException.scala: -------------------------------------------------------------------------------- 1 | /** 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 | 18 | package org.dist.queue.common 19 | 20 | /** 21 | * Thrown when a request is made for partition, but no leader exists for that partition 22 | */ 23 | class ReplicaNotAvailableException(cause: Throwable, message: String = "") extends RuntimeException(cause) { 24 | def this() = this(null, "") 25 | def this(message: String) = this(null, message) 26 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/StateChangeFailedException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class StateChangeFailedException(message: String, cause: Throwable) extends RuntimeException(message, cause) { 4 | def this(message: String) = this(message, null) 5 | def this() = this(null, null) 6 | } 7 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/Topic.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | object Topic { 4 | val GROUP_METADATA_TOPIC_NAME = "__consumer_offsets" 5 | val groupMetadataTopicPartitionCount = 3 6 | } 7 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/TopicAndPartition.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class TopicAndPartition(topic: String, partition: Int) { 4 | 5 | def this(tuple: (String, Int)) = this(tuple._1, tuple._2) 6 | 7 | def asTuple = (topic, partition) 8 | 9 | override def toString = "[%s,%d]".format(topic, partition) 10 | } 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/common/UnknownTopicOrPartitionException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.common 2 | 3 | case class UnknownTopicOrPartitionException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/controller/AddPartitionsListener.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.controller 2 | 3 | import org.I0Itec.zkclient.IZkDataListener 4 | import org.dist.queue.common.Logging 5 | import org.dist.queue.utils.ZkUtils 6 | 7 | class AddPartitionsListener(topic: String, controller:Controller) extends IZkDataListener with Logging { 8 | private val controllerContext = controller.controllerContext 9 | private val zkClient = controllerContext.zkClient 10 | 11 | this.logIdent = "[AddPartitionsListener on " + controller.config.brokerId + "]: " 12 | 13 | @throws(classOf[Exception]) 14 | def handleDataChange(dataPath: String, data: Object) { 15 | controllerContext.controllerLock synchronized { 16 | try { 17 | info("Add Partition triggered " + data.toString + " for path " + dataPath) 18 | val partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic)) 19 | val partitionsRemainingToBeAdded = partitionReplicaAssignment.filter(p => 20 | !controllerContext.partitionReplicaAssignment.contains(p._1)) 21 | info("New partitions to be added [%s]".format(partitionsRemainingToBeAdded)) 22 | controller.onNewPartitionCreation(partitionsRemainingToBeAdded.keySet.toSet) 23 | } catch { 24 | case e: Throwable => error("Error while handling add partitions for data path " + dataPath, e) 25 | } 26 | } 27 | } 28 | 29 | @throws(classOf[Exception]) 30 | def handleDataDeleted(parentPath: String) { 31 | // this is not implemented for partition change 32 | } 33 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/controller/OfflinePartitionLeaderSelector.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.controller 2 | 3 | import org.dist.queue.common.{Logging, TopicAndPartition} 4 | 5 | 6 | class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) extends PartitionLeaderSelector with Logging { 7 | /** 8 | * @param topicAndPartition The topic and partition whose leader needs to be elected 9 | * @param currentLeaderAndIsr The current leader and isr of input partition read from zookeeper 10 | * @throws NoReplicaOnlineException If no replica in the assigned replicas list is alive 11 | * @return The leader and isr request, with the newly selected leader info, to send to the brokers 12 | * Also, returns the list of replicas the returned leader and isr request should be sent to 13 | * This API selects a new leader for the input partition 14 | */ 15 | override def selectLeader(topicAndPartition: TopicAndPartition, currentLeaderAndIsr: LeaderAndIsr): (LeaderAndIsr, Seq[Int]) = { 16 | (LeaderAndIsr(1, 1, List(), 1), Seq[Int](1)) 17 | } 18 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/coordinator/GroupMetadataManager.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.coordinator 2 | 3 | import org.dist.queue.common.Topic 4 | import org.dist.queue.server.{Config, KafkaZooKeeper, ReplicaManager} 5 | 6 | class GroupMetadataManager(replicaManager: ReplicaManager, 7 | kafkaZooKeeper: KafkaZooKeeper, 8 | config:Config) { 9 | private def getGroupMetadataTopicPartitionCount: Int = { 10 | kafkaZooKeeper.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME).getOrElse(config.offsetsTopicNumPartitions) 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/InvalidMessageException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.log 2 | 3 | case class InvalidMessageException() extends RuntimeException{ 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/InvalidOffsetException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.log 2 | 3 | case class InvalidOffsetException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/KafkaTimer.scala: -------------------------------------------------------------------------------- 1 | /** 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 | 18 | package org.dist.queue.log 19 | 20 | import com.yammer.metrics.core.Timer 21 | 22 | /** 23 | * A wrapper around metrics timer object that provides a convenient mechanism 24 | * to time code blocks. This pattern was borrowed from the metrics-scala_2.9.1 25 | * package. 26 | * @param metric The underlying timer object. 27 | */ 28 | class KafkaTimer(metric: Timer) { 29 | 30 | def time[A](f: => A): A = { 31 | val ctx = metric.time 32 | try { 33 | f 34 | } 35 | finally { 36 | ctx.stop() 37 | } 38 | } 39 | } 40 | 41 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/OffsetOutOfRangeException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.log 2 | 3 | case class OffsetOutOfRangeException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/OffsetPosition.scala: -------------------------------------------------------------------------------- 1 | /** 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 | 18 | package org.dist.queue.log 19 | 20 | /** 21 | * The mapping between a logical log offset and the physical position 22 | * in some log file of the beginning of the message set entry with the 23 | * given offset. 24 | */ 25 | case class OffsetPosition(val offset: Long, val position: Int) -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/log/Range.scala: -------------------------------------------------------------------------------- 1 | /** 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 | 18 | package org.dist.queue.log 19 | 20 | /** 21 | * A generic range value with a start and end 22 | */ 23 | trait Range { 24 | /** The first index in the range */ 25 | def start: Long 26 | /** The total number of indexes in the range */ 27 | def size: Long 28 | /** Return true iff the range is empty */ 29 | def isEmpty: Boolean = size == 0 30 | 31 | /** if value is in range */ 32 | def contains(value: Long): Boolean = { 33 | if( (size == 0 && value == start) || 34 | (size > 0 && value >= start && value <= start + size - 1) ) 35 | return true 36 | else 37 | return false 38 | } 39 | 40 | override def toString() = "(start=" + start + ", size=" + size + ")" 41 | } 42 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/message/CompressionFactory.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.message 2 | 3 | import java.io.{InputStream, OutputStream} 4 | import java.util.zip.{GZIPInputStream, GZIPOutputStream} 5 | 6 | object CompressionFactory { 7 | 8 | def apply(compressionCodec: CompressionCodec, stream: OutputStream): OutputStream = { 9 | compressionCodec match { 10 | case DefaultCompressionCodec => new GZIPOutputStream(stream) 11 | case GZIPCompressionCodec => new GZIPOutputStream(stream) 12 | // case SnappyCompressionCodec => 13 | // 14 | // new SnappyOutputStream(stream) 15 | // case _ => 16 | throw new RuntimeException("Unknown Codec: " + compressionCodec) 17 | } 18 | } 19 | 20 | def apply(compressionCodec: CompressionCodec, stream: InputStream): InputStream = { 21 | compressionCodec match { 22 | case DefaultCompressionCodec => new GZIPInputStream(stream) 23 | case GZIPCompressionCodec => new GZIPInputStream(stream) 24 | // case SnappyCompressionCodec => 25 | // 26 | // new SnappyInputStream(stream) 27 | case _ => 28 | throw new RuntimeException("Unknown Codec: " + compressionCodec) 29 | } 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/message/KeyedMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.message 2 | 3 | /** 4 | * A topic, key, and value 5 | */ 6 | case class KeyedMessage[K, V](val topic: String, val key: K, val message: V) { 7 | if(topic == null) 8 | throw new IllegalArgumentException("Topic cannot be null.") 9 | 10 | def this(topic: String, message: V) = this(topic, null.asInstanceOf[K], message) 11 | 12 | def hasKey = key != null 13 | } 14 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/message/MessageAndOffset.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.message 2 | 3 | case class MessageAndOffset(message: Message, offset: Long) { 4 | 5 | /** 6 | * Compute the offset of the next message in the log 7 | */ 8 | def nextOffset: Long = offset + 1 9 | } 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/message/MessageSizeTooLargeException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.message 2 | 3 | case class MessageSizeTooLargeException(str: String) extends RuntimeException(str) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/network/Client.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.network 2 | 3 | import java.util.concurrent.atomic.AtomicInteger 4 | 5 | import org.dist.kvstore.{InetAddressAndPort, JsonSerDes} 6 | import org.dist.queue.api.{RequestKeys, RequestOrResponse, TopicMetadataRequest, TopicMetadataResponse} 7 | import org.dist.queue.server.Config 8 | 9 | import scala.collection.Set 10 | 11 | class Client(bootstrapBroker:InetAddressAndPort, config:Config) { 12 | val correlationId = new AtomicInteger(0) 13 | val clientId = "client1" 14 | val socketClient = new SocketClient 15 | 16 | 17 | def fetchTopicMetadata(topics: Set[String]): Unit = { 18 | val correlationIdForRequest = correlationId.getAndIncrement() 19 | val topicMetadataRequest = new TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, correlationIdForRequest, clientId, topics.toSeq) 20 | val response = socketClient.sendReceiveTcp(new RequestOrResponse(RequestKeys.GetMetadataKey, JsonSerDes.serialize(topicMetadataRequest), correlationIdForRequest), bootstrapBroker) 21 | val topicMetadataResponse = JsonSerDes.deserialize(response.messageBodyJson.getBytes(), classOf[TopicMetadataResponse]) 22 | print(topicMetadataResponse.topicsMetadata) 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/queue/network/SocketClient.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.network 2 | 3 | import java.net.Socket 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | import org.dist.queue.api.RequestOrResponse 7 | import org.dist.util.SocketIO 8 | 9 | class SocketClient { 10 | 11 | def sendReceiveTcp(message: RequestOrResponse, to: InetAddressAndPort) = { 12 | val clientSocket = new Socket(to.address, to.port) 13 | new SocketIO[RequestOrResponse](clientSocket, classOf[RequestOrResponse]).requestResponse(message) 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/MembershipView.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid 2 | 3 | import java.util 4 | import java.util.Collection 5 | 6 | import org.dist.kvstore.InetAddressAndPort 7 | 8 | case class MembershipView(endpoints: util.List[InetAddressAndPort]) { 9 | sort 10 | 11 | def addEndpoint(address: InetAddressAndPort) = { 12 | if (!endpoints.contains(address)) { 13 | endpoints.add(address) 14 | sort 15 | } 16 | } 17 | 18 | private def sort = { 19 | endpoints.sort((a1, a2) => { 20 | a1.hashCode().compareTo(a2.hashCode()) 21 | }) 22 | } 23 | 24 | def getObservers() = endpoints.get(0) 25 | 26 | def isSafeToJoin() = true 27 | 28 | private val K: Int = 10 29 | 30 | } 31 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/SocketClient.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid 2 | 3 | import java.net.Socket 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | import org.dist.queue.api.RequestOrResponse 7 | import org.dist.util.SocketIO 8 | 9 | class SocketClient { 10 | def sendReceive(requestOrResponse: RequestOrResponse, to:InetAddressAndPort):RequestOrResponse = { 11 | val clientSocket = new Socket(to.address, to.port) 12 | new SocketIO[RequestOrResponse](clientSocket, classOf[RequestOrResponse]).requestResponse(requestOrResponse) 13 | } 14 | 15 | def sendOneWay(requestOrResponse: RequestOrResponse, to:InetAddressAndPort):Unit = { 16 | val clientSocket = new Socket(to.address, to.port) 17 | new SocketIO[RequestOrResponse](clientSocket, classOf[RequestOrResponse]).write(requestOrResponse) 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/AlertMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | object AlertMessage { 6 | val UP = 0 7 | val DOWN = 1 8 | } 9 | 10 | case class AlertMessage(status:Int, address:InetAddressAndPort) 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/JoinMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | case class JoinMessage(address:InetAddressAndPort) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/JoinResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | case class JoinResponse(endPoints:List[InetAddressAndPort]) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Phase1aMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | case class Phase1aMessage(configurationId:Int, address:InetAddressAndPort, rank:Rank) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Phase1bMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | import java.util 5 | 6 | case class Phase1bMessage(configurationId:Int, 7 | rnd:Rank, 8 | sender:InetAddressAndPort, 9 | vrnd:Rank, 10 | vval:util.List[InetAddressAndPort]) 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Phase2aMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import java.util 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | 7 | case class Phase2aMessage(configurationId:Int, 8 | sender:InetAddressAndPort, 9 | crnd:Rank, 10 | chosenProposal:util.List[InetAddressAndPort]) 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Phase2bMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import java.util 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | 7 | case class Phase2bMessage(configurationId:Int, rnd:Rank, sender:InetAddressAndPort, vval: util.List[InetAddressAndPort]) 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/PreJoinMessage.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | case class PreJoinMessage(configId:Int, nodeId:String) { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Proposal.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | import org.dist.kvstore.InetAddressAndPort 4 | 5 | case class Proposal(endpoint:InetAddressAndPort) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/Rank.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | case class Rank(round:Int, nodeIndex:Int) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/rapid/messages/RapidMessages.scala: -------------------------------------------------------------------------------- 1 | package org.dist.rapid.messages 2 | 3 | object RapidMessages { 4 | val preJoinMessage:Short = 0 5 | val joinMessage:Short = 1 6 | val joinPhase1Response:Short = 2 7 | val alertMessage:Short = 3 8 | val fastPaxosMessage:Short = 4 9 | val phase1aMessage:Short = 5 10 | val phase1bMessage:Short = 6 11 | val phase2aMessage: Short = 7 12 | val phase2bMessage: Short = 8 13 | } 14 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/BrokerChangeListner.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import java.util 4 | 5 | import org.I0Itec.zkclient.IZkChildListener 6 | import org.dist.queue.common.Logging 7 | 8 | class BrokerChangeListener(controller:Controller, zookeeperClient:ZookeeperClient) extends IZkChildListener with Logging { 9 | this.logIdent = "[BrokerChangeListener on Controller " + controller.brokerId + "]: " 10 | 11 | import scala.jdk.CollectionConverters._ 12 | 13 | override def handleChildChange(parentPath: String, currentBrokerList: util.List[String]): Unit = { 14 | info("Broker change listener fired for path %s with children %s".format(parentPath, currentBrokerList.asScala.mkString(","))) 15 | try { 16 | 17 | val curBrokerIds = currentBrokerList.asScala.map(_.toInt).toSet 18 | val newBrokerIds = curBrokerIds -- controller.liveBrokers.map(broker => broker.id) 19 | val newBrokers = newBrokerIds.map(zookeeperClient.getBrokerInfo(_)) 20 | 21 | newBrokers.foreach(controller.addBroker(_)) 22 | 23 | if (newBrokerIds.size > 0) 24 | controller.onBrokerStartup(newBrokerIds.toSeq) 25 | 26 | } catch { 27 | case e: Throwable => error("Error while handling broker changes", e) 28 | } 29 | } 30 | } -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/ConsumeRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.common.TopicAndPartition 4 | 5 | case class ConsumeRequest(topicAndPartition: TopicAndPartition, offset:Int = 0) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/ConsumeResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | case class ConsumeResponse(messages:Map[String, String]) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/LeaderAndReplicas.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.common.TopicAndPartition 4 | import org.dist.queue.utils.ZkUtils.Broker 5 | 6 | case class PartitionInfo(leader:Broker, allReplicas:List[Broker]) 7 | 8 | case class LeaderAndReplicas(topicPartition:TopicAndPartition, partitionStateInfo:PartitionInfo) 9 | 10 | case class LeaderAndReplicaRequest(leaderReplicas:List[LeaderAndReplicas]) 11 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/ProduceRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.common.TopicAndPartition 4 | 5 | case class ProduceRequest(topicAndPartition: TopicAndPartition, key:String, message:String) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/ReplicaAssignmentStrategy.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | class ReplicaAssignmentStrategy { 4 | def assignReplica(partitionId: Int, replicationFactor: Int, brokerIds: List[Int]): List[Int] = { 5 | brokerIds.slice(0, replicationFactor) 6 | } 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/ReplicaManager.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import java.util 4 | 5 | import akka.actor.ActorSystem 6 | import org.dist.queue.common.TopicAndPartition 7 | import org.dist.queue.server.Config 8 | 9 | class ReplicaManager(config:Config)(implicit actorSystem:ActorSystem) { 10 | 11 | val allPartitions = new util.HashMap[TopicAndPartition, Partition]() 12 | 13 | def makeFollower(topicAndPartition: TopicAndPartition, leaderId:Int) = { 14 | val partition = getOrCreatePartition(topicAndPartition) 15 | partition.makeFollower(leaderId) 16 | } 17 | 18 | def makeLeader(topicAndPartition: TopicAndPartition) = { 19 | val partition = getOrCreatePartition(topicAndPartition) 20 | partition.makeLeader() 21 | } 22 | 23 | def getPartition(topicAndPartition: TopicAndPartition) = { 24 | allPartitions.get(topicAndPartition) 25 | } 26 | 27 | def getOrCreatePartition(topicAndPartition: TopicAndPartition) = { 28 | var partition = allPartitions.get(topicAndPartition) 29 | if (null == partition) { 30 | partition = new Partition(config, topicAndPartition) 31 | allPartitions.put(topicAndPartition, partition) 32 | } 33 | partition 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/Server.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.common.Logging 4 | import org.dist.queue.server.Config 5 | 6 | class Server(val config:Config, val zookeeperClient: ZookeeperClient, val controller:Controller, val socketServer: SimpleSocketServer) extends Logging { 7 | def startup() = { 8 | socketServer.startup() 9 | zookeeperClient.registerSelf() 10 | controller.startup() 11 | 12 | info(s"Server ${config.brokerId} started with log dir ${config.logDirs}") 13 | } 14 | 15 | def shutdown()= { 16 | zookeeperClient.shutdown() 17 | socketServer.shutdown() 18 | } 19 | } 20 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/SocketClient.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import java.net.Socket 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | import org.dist.queue.api.RequestOrResponse 7 | import org.dist.util.SocketIO 8 | 9 | class SocketClient { 10 | 11 | def sendReceiveTcp(message: RequestOrResponse, to: InetAddressAndPort) = { 12 | val clientSocket = new Socket(to.address, to.port) 13 | new SocketIO[RequestOrResponse](clientSocket, classOf[RequestOrResponse]).requestResponse(message) 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/TimeoutException.java: -------------------------------------------------------------------------------- 1 | /* 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 | package org.dist.simplekafka; 18 | 19 | /** 20 | * Indicates that a request timed out. 21 | */ 22 | public class TimeoutException extends RuntimeException { 23 | 24 | private static final long serialVersionUID = 1L; 25 | 26 | public TimeoutException() { 27 | super(); 28 | } 29 | 30 | public TimeoutException(String message, Throwable cause) { 31 | super(message, cause); 32 | } 33 | 34 | public TimeoutException(String message) { 35 | super(message); 36 | } 37 | 38 | public TimeoutException(Throwable cause) { 39 | super(cause); 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/TopicChangeHandler.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import java.util 4 | 5 | import org.I0Itec.zkclient.IZkChildListener 6 | import org.dist.queue.common.Logging 7 | 8 | import scala.jdk.CollectionConverters._ 9 | 10 | class TopicChangeHandler(zookeeperClient:ZookeeperClient, onTopicChange:(String, Seq[PartitionReplicas]) => Unit) extends IZkChildListener with Logging { 11 | override def handleChildChange(parentPath: String, currentChilds: util.List[String]): Unit = { 12 | currentChilds.asScala.foreach(topicName => { 13 | val replicas: Seq[PartitionReplicas] = zookeeperClient.getPartitionAssignmentsFor(topicName) 14 | onTopicChange(topicName, replicas) 15 | }) 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/TopicMetadataRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | case class TopicMetadataRequest(topicName:String) 4 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/TopicMetadataResponse.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.common.TopicAndPartition 4 | 5 | case class TopicMetadataResponse(topicPartitions:Map[TopicAndPartition, PartitionInfo]) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/UpdateMetadataRequest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.utils.ZkUtils.Broker 4 | 5 | case class UpdateMetadataRequest(aliveBrokers:List[Broker], leaderReplicas:List[LeaderAndReplicas]) 6 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/utils/timer/Time.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka.utils.timer 2 | 3 | /** 4 | * A mockable interface for time functions 5 | */ 6 | trait Time { 7 | 8 | def milliseconds: Long 9 | 10 | def nanoseconds: Long 11 | 12 | def sleep(ms: Long) 13 | } 14 | 15 | /** 16 | * The normal system implementation of time functions 17 | */ 18 | object SystemTime extends Time { 19 | 20 | def milliseconds: Long = System.currentTimeMillis 21 | 22 | def nanoseconds: Long = System.nanoTime 23 | 24 | def sleep(ms: Long): Unit = Thread.sleep(ms) 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/simplekafka/utils/timer/TimerTask.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka.utils.timer 2 | 3 | trait TimerTask extends Runnable { 4 | 5 | val delayMs: Long // timestamp in millisecond 6 | 7 | private var timerTaskEntry: TimerTaskEntry = null 8 | 9 | def cancel(): Unit = { 10 | synchronized { 11 | if (timerTaskEntry != null) timerTaskEntry.remove() 12 | timerTaskEntry = null 13 | } 14 | } 15 | 16 | def setTimerTaskEntry(entry: TimerTaskEntry): Unit = { 17 | synchronized { 18 | // if this timerTask is already held by an existing timer task entry, 19 | // we will remove such an entry first. 20 | if (timerTaskEntry != null && timerTaskEntry != entry) 21 | timerTaskEntry.remove() 22 | 23 | timerTaskEntry = entry 24 | } 25 | } 26 | 27 | def getTimerTaskEntry: TimerTaskEntry = timerTaskEntry 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/ByteUtils.java: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore; 2 | 3 | import org.apache.commons.codec.binary.Hex; 4 | 5 | public class ByteUtils { 6 | public static String toHexString(byte[] key) { 7 | return Hex.encodeHexString(key); 8 | } 9 | } 10 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/HashFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2008-2009 LinkedIn, Inc 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not 5 | * use this file except in compliance with the License. You may obtain a copy of 6 | * 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, WITHOUT 12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 13 | * License for the specific language governing permissions and limitations under 14 | * the License. 15 | */ 16 | 17 | package org.dist.versionedkvstore; 18 | 19 | /** 20 | * A hash function for bytes, deterministically maps bytes into ints 21 | * 22 | * 23 | */ 24 | public interface HashFunction { 25 | 26 | public int hash(byte[] key); 27 | 28 | } 29 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/InconsistentDataException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore 2 | 3 | import java.util 4 | 5 | class InconsistentDataException(val message: String, var versions: util.List[Versioned[_]]) extends RuntimeException(message) { 6 | def getVersions = this.versions 7 | } 8 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/InvalidClockEntryException.java: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore; 2 | 3 | 4 | public class InvalidClockEntryException extends RuntimeException { 5 | 6 | private static final long serialVersionUID = 1L; 7 | 8 | public InvalidClockEntryException() { 9 | super(); 10 | } 11 | 12 | public InvalidClockEntryException(String s, Throwable t) { 13 | super(s, t); 14 | } 15 | 16 | public InvalidClockEntryException(String s) { 17 | super(s); 18 | } 19 | 20 | public InvalidClockEntryException(Throwable t) { 21 | super(t); 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/Node.scala: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore 2 | 3 | import java.util 4 | import java.util.{Collections, List} 5 | 6 | //represents server 7 | class Node[K, V](val id:Int, 8 | partitions: util.List[Int] = Collections.emptyList(), 9 | host: String = "localhost", 10 | httpPort: Int = 0, 11 | socketPort: Int = 0, 12 | adminPort: Int = 0) { 13 | 14 | 15 | def put(key: K, versionedCopy: Versioned[V]) = { 16 | map.put(key, versionedCopy) 17 | } 18 | 19 | val map = new VersionedMap[K, V]() 20 | def get(key:K) = map.get(key) 21 | def getVersions(key:K) = map.getVersions(key) 22 | 23 | import scala.jdk.CollectionConverters._ 24 | def getPartitionIds():util.List[Int] = partitions.asScala.asJava 25 | } 26 | 27 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/ObsoleteVersionException.scala: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore 2 | 3 | /** 4 | * An exception that indicates an attempt by the user to overwrite a newer value 5 | * for a given key with an older value for the same key. This is a 6 | * application-level error, and indicates the application has attempted to write 7 | * stale data. 8 | * 9 | * 10 | */ 11 | class ObsoleteVersionException(message: String, cause: Exception) extends RuntimeException(message, cause) { 12 | def this(message: String ) { 13 | this(message, null) 14 | } 15 | } 16 | 17 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/Occurred.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2008-2009 LinkedIn, Inc 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not 5 | * use this file except in compliance with the License. You may obtain a copy of 6 | * 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, WITHOUT 12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 13 | * License for the specific language governing permissions and limitations under 14 | * the License. 15 | */ 16 | 17 | package org.dist.versionedkvstore; 18 | 19 | /** 20 | * The result of comparing two times--either t1 is BEFORE t2, t1 is AFTER t2, or 21 | * t1 happens CONCURRENTLY to t2. 22 | * 23 | * 24 | */ 25 | public enum Occurred { 26 | BEFORE, 27 | AFTER, 28 | CONCURRENTLY 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/RoutingStrategyType.java: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore; 2 | 3 | /** 4 | * An enumeration of RoutingStrategies type 5 | * 6 | * 7 | */ 8 | public class RoutingStrategyType { 9 | 10 | public final static String CONSISTENT_STRATEGY = "consistent-routing"; 11 | public final static String TO_ALL_STRATEGY = "all-routing"; 12 | public final static String ZONE_STRATEGY = "zone-routing"; 13 | public final static String TO_ALL_LOCAL_PREF_STRATEGY = "local-pref-all-routing"; 14 | 15 | private final String name; 16 | 17 | private RoutingStrategyType(String name) { 18 | this.name = name; 19 | } 20 | 21 | public String getName() { 22 | return name; 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/Version.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2008-2009 LinkedIn, Inc 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); you may not 5 | * use this file except in compliance with the License. You may obtain a copy of 6 | * 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, WITHOUT 12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 13 | * License for the specific language governing permissions and limitations under 14 | * the License. 15 | */ 16 | 17 | package org.dist.versionedkvstore; 18 | 19 | /** 20 | * An interface that allows us to determine if a given version happened before 21 | * or after another version. 22 | * 23 | * This could have been done using the comparable interface but that is 24 | * confusing, because the numeric codes are easily confused, and because 25 | * concurrent versions are not necessarily "equal" in the normal sense. 26 | * 27 | * 28 | */ 29 | public interface Version { 30 | 31 | /** 32 | * Return whether or not the given version preceded this one, succeeded it, 33 | * or is concurrent with it 34 | * 35 | * @param v The other version 36 | */ 37 | public Occurred compare(Version v); 38 | 39 | } 40 | -------------------------------------------------------------------------------- /src/main/scala/org/dist/versionedkvstore/Versioned.scala: -------------------------------------------------------------------------------- 1 | package org.dist.versionedkvstore 2 | 3 | import java.io.Serializable 4 | import java.util.Comparator 5 | 6 | @SerialVersionUID(1) 7 | object Versioned { 8 | def value[S](s: S) = new Versioned[S](s, new VectorClock) 9 | 10 | def value[S](s: S, v: Version) = new Versioned[S](s, v) 11 | 12 | final class HappenedBeforeComparator[S] extends Comparator[Versioned[S]] { 13 | def compare(v1: Versioned[S], v2: Versioned[S]): Int = { 14 | val occurred = v1.getVersion.compare(v2.getVersion) 15 | if (occurred eq Occurred.BEFORE) -1 16 | else if (occurred eq Occurred.AFTER) 1 17 | else 0 18 | } 19 | } 20 | 21 | } 22 | 23 | @SerialVersionUID(1) 24 | case class Versioned[T](var value: T, val version: Version) extends Serializable { 25 | 26 | def this(value: T) { 27 | this(value, new VectorClock) 28 | } 29 | 30 | def getVersion: Version = version 31 | 32 | def getValue: T = value 33 | 34 | def setObject(value: T): Unit = { 35 | this.value = value 36 | } 37 | } 38 | 39 | -------------------------------------------------------------------------------- /src/test/java/org/dist/patterns/ignite/cluster/TcpDiscoveryNodeTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.ignite.cluster; 2 | 3 | import org.junit.Test; 4 | 5 | import static org.junit.Assert.*; 6 | 7 | public class TcpDiscoveryNodeTest { 8 | 9 | @Test 10 | public void singleServerShouldMarkItSelfAsCoordinator() { 11 | new TcpDiscoveryNode(); 12 | } 13 | 14 | } -------------------------------------------------------------------------------- /src/test/java/org/dist/patterns/session/ExpiryQueueTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.session; 2 | 3 | import org.apache.zookeeper.common.Time; 4 | import org.junit.Test; 5 | 6 | import java.util.Set; 7 | 8 | import static org.junit.Assert.*; 9 | 10 | public class ExpiryQueueTest { 11 | 12 | @Test 13 | public void shouldSetExpirationTimeForGivenKey() throws InterruptedException { 14 | ExpiryQueue queue = new ExpiryQueue(3000); 15 | 16 | queue.update("sessionId", 1000, Time.currentElapsedTime()); 17 | Thread.sleep(1000); 18 | queue.update("sessionId1", 1000, Time.currentElapsedTime()); 19 | queue.update("sessionId3", 1000, Time.currentElapsedTime()); 20 | var now = Time.currentElapsedTime(); 21 | long timeout = ((now + 1000) / 3000 + 1) * 3000; 22 | 23 | Set o = (Set) queue.getExpiryMap().get(timeout); 24 | System.out.println(o.size()); 25 | assertEquals(o.iterator().next(), "sessionId"); 26 | } 27 | @Test 28 | public void shouldExpireElementsAfterTimeout() { 29 | ExpiryQueue queue = new ExpiryQueue(100); 30 | queue.update("sessionId", 1000, Time.currentElapsedTime()); 31 | 32 | Set expiredElements = queue.poll(); 33 | while(expiredElements.size() == 0) { 34 | expiredElements = queue.poll(); 35 | } 36 | assertEquals(expiredElements.size(), 1); 37 | } 38 | } -------------------------------------------------------------------------------- /src/test/java/org/dist/patterns/session/SessionTrackerImplTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.session; 2 | 3 | import org.dist.queue.TestUtils; 4 | import org.junit.Test; 5 | 6 | import java.time.Duration; 7 | 8 | import static org.junit.Assert.*; 9 | 10 | public class SessionTrackerImplTest { 11 | @Test 12 | public void testSessionTracker() throws InterruptedException { 13 | SessionTrackerImpl sessionTracker = new SessionTrackerImpl(3000, 1); 14 | sessionTracker.start(); 15 | long session = sessionTracker.createSession(1000); 16 | sessionTracker.touchSession(session, 1000); 17 | TestUtils.waitUntilTrue(()-> sessionTracker.getSessionsById().size() == 0, 18 | ()->"waiting for session to expire", 1000L, 100); 19 | } 20 | } -------------------------------------------------------------------------------- /src/test/java/org/dist/utils/JTestUtils.java: -------------------------------------------------------------------------------- 1 | package org.dist.utils; 2 | 3 | import java.io.File; 4 | import java.io.IOException; 5 | import java.net.ServerSocket; 6 | import java.util.ArrayList; 7 | import java.util.List; 8 | import java.util.Random; 9 | import java.util.stream.Collectors; 10 | 11 | public class JTestUtils { 12 | public static List choosePorts(Integer count) throws IOException { 13 | List serverSockets = new ArrayList<>(); 14 | for (int i = 0; i < count; i++) { 15 | serverSockets.add(new ServerSocket(0)); 16 | } 17 | List ports = serverSockets.stream().map(s -> { 18 | int localPort = s.getLocalPort(); 19 | try { 20 | s.close(); 21 | } catch (IOException e) { 22 | throw new RuntimeException(e); 23 | } 24 | return localPort; 25 | }).collect(Collectors.toList()); 26 | 27 | return ports; 28 | } 29 | 30 | public static Integer choosePort() throws IOException { 31 | return choosePorts(1).get(0); 32 | } 33 | 34 | static Random random = new Random(); 35 | 36 | public static File tmpDir(String prefix) { 37 | var ioDir = System.getProperty("java.io.tmpdir"); 38 | var f = new File(ioDir, prefix + random.nextInt(1000000)); 39 | f.mkdirs(); 40 | f.deleteOnExit(); 41 | return f; 42 | } 43 | 44 | } 45 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/consensus/raft/Log.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.raft 2 | 3 | class Log { 4 | 5 | } 6 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/consensus/zab/TransactionSerializationTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.io.ByteArrayInputStream 4 | 5 | import org.scalatest.FunSuite 6 | 7 | class TransactionSerializationTest extends FunSuite { 8 | 9 | test("should serialize and deserialize Transactions") { 10 | val request = Request(null, 1, 1, Array[Byte](2)) 11 | request.txnHeader = TxnHeader(request.sessionId, request.xid, 1000, 1000, OpsCode.setData) 12 | request.txn = SetDataTxn("path", request.data) 13 | 14 | val bytes = request.serializeTxn() 15 | 16 | val tuple = Request.deserializeTxn(new ByteArrayInputStream(bytes)) 17 | assert(tuple._1 == request.txnHeader) 18 | assert(tuple._2.path == request.txn.path) 19 | assert(tuple._2.version == request.txn.version) 20 | assert(tuple._2.data.sameElements(request.txn.data)) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/consensus/zab/ZabTestUtils.scala: -------------------------------------------------------------------------------- 1 | package org.dist.consensus.zab 2 | 3 | import java.io.File 4 | 5 | import org.dist.queue.TestUtils.random 6 | 7 | object ZabTestUtils { 8 | def tempDir(): File = { 9 | val ioDir = System.getProperty("java.io.tmpdir") 10 | val f = new File(ioDir, "zab-" + random.nextInt(1000000)) 11 | f.mkdirs() 12 | f.deleteOnExit() 13 | f 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/DbManagerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.io.File 4 | import java.nio.file.Paths 5 | 6 | import org.scalatest.{BeforeAndAfter, FunSuite} 7 | 8 | class DbManagerTest extends FunSuite with BeforeAndAfter { 9 | 10 | before { 11 | TestUtils.deleteTable("system") 12 | } 13 | 14 | after { 15 | TestUtils.deleteTable("system") 16 | } 17 | 18 | 19 | test("should generate new token and generation info if its not already saved") { 20 | val metadata = new DbManager(TestUtils.tmpDir.toString).start(InetAddressAndPort.create("127.7.7.1", 8888)) 21 | assert(metadata.generation == 1) 22 | assert(metadata.token != null) 23 | } 24 | 25 | test("should use existing token and increment generation info if its already saved") { 26 | //start to create a file 27 | new DbManager(TestUtils.tmpDir.toString).start(InetAddressAndPort.create("127.7.7.1", 8888)) 28 | 29 | val metadata = new DbManager(TestUtils.tmpDir.toString).start(InetAddressAndPort.create("127.7.7.1", 8888)) 30 | assert(metadata.generation == 2) 31 | assert(metadata.token != null) 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/GossiperTaskTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import org.scalatest.FunSuite 4 | import scala.jdk.CollectionConverters._ 5 | 6 | class GossiperTaskTest extends FunSuite { 7 | 8 | test("Should update heartbeat at every run") { 9 | val seeds = Set(InetAddressAndPort.create("127.0.0.1", 8000)) 10 | val executor = new TestScheduledThreadPoolExecutor 11 | val messagingService = new TestMessagingService 12 | 13 | val localEp = InetAddressAndPort.create("127.0.0.1", 8000) 14 | val gossiper = new Gossiper(1, localEp, 15 | DatabaseConfiguration(seeds), executor, messagingService) 16 | 17 | val task = new gossiper.GossipTask() 18 | task.run() 19 | task.run() 20 | 21 | val state = gossiper.endpointStatemap.get(localEp) 22 | assert(state.heartBeatState.heartBeat.get() == 2) 23 | } 24 | 25 | test("Should update heartbeat version when heartbeat is updated") { 26 | val seeds = Set(InetAddressAndPort.create("127.0.0.1", 8000)) 27 | val executor = new TestScheduledThreadPoolExecutor 28 | val messagingService = new TestMessagingService 29 | 30 | val localEp = InetAddressAndPort.create("127.0.0.1", 8000) 31 | val gossiper = new Gossiper(1, localEp, 32 | DatabaseConfiguration(seeds), executor, messagingService) 33 | 34 | val task = new gossiper.GossipTask() 35 | task.run() 36 | task.run() 37 | 38 | val state = gossiper.endpointStatemap.get(localEp) 39 | assert(state.heartBeatState.version == 2) 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/GossiperTestBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.util.concurrent.ScheduledThreadPoolExecutor 4 | import scala.jdk.CollectionConverters._ 5 | 6 | 7 | class GossiperTestBuilder(val seeds:Set[InetAddressAndPort] = Set(InetAddressAndPort.create("127.0.0.1", 8000)), 8 | val localEndpoint:InetAddressAndPort = InetAddressAndPort.create("127.0.0.1", 8000), 9 | val executor:ScheduledThreadPoolExecutor = new TestScheduledThreadPoolExecutor, 10 | val messagingService:MessagingService = new TestMessagingService) { 11 | def build() = gossiper 12 | 13 | 14 | private val gossiper = new Gossiper(1, localEndpoint, DatabaseConfiguration(seeds), executor, messagingService) 15 | 16 | def withEndpointState(host: String, port: Int, hearBeatVersion: Int = 0, applicationStates: Map[ApplicationState, VersionedValue], generation: Int) = { 17 | val ep = EndPointState(HeartBeatState(generation, hearBeatVersion), applicationStates.asJava) 18 | gossiper.endpointStatemap.put(InetAddressAndPort.create(host, port), ep) 19 | this 20 | } 21 | 22 | def withLiveEndpoints(hosts:List[(String, Int)]) = { 23 | hosts.map(hostPort => gossiper.liveEndpoints.add(InetAddressAndPort.create(hostPort._1, hostPort._2))) 24 | this 25 | } 26 | 27 | def withUnReachableEndpoints(hosts:List[(String, Int)]) = { 28 | hosts.map(hostPort => gossiper.unreachableEndpoints.add(InetAddressAndPort.create(hostPort._1, hostPort._2))) 29 | this 30 | } 31 | } 32 | 33 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/JsonSerDesTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class JsonSerDesTest extends FunSuite { 6 | 7 | test("should be able to serialize and deserialize Gossip messages") { 8 | val messageJson = "{\"header\":{\"from\":{\"address\":\"192.168.0.120\",\"port\":8000,\"defaultPort\":7000},\"messageType\":\"GOSSIP\",\"verb\":\"GOSSIP_DIGEST_ACK\"},\"payloadJson\":\"{\\\"gDigestList\\\":[{\\\"endPoint\\\":{\\\"address\\\":\\\"192.168.0.120\\\",\\\"port\\\":8001,\\\"defaultPort\\\":7000},\\\"generation\\\":1,\\\"maxVersion\\\":0}],\\\"epStateMap\\\":{}}\"}" 9 | val message = JsonSerDes.deserialize(messageJson.getBytes, classOf[Message]) 10 | assert(message != null) 11 | val ack = JsonSerDes.deserialize(message.payloadJson.getBytes, classOf[GossipDigestAck]) 12 | assert(ack != null) 13 | } 14 | 15 | } 16 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/TableReadWriteTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import org.dist.queue.TestUtils 4 | import org.scalatest.FunSuite 5 | 6 | class TableReadWriteTest extends FunSuite { 7 | 8 | test("should write and read rows from table") { 9 | val table = new Table(TestUtils.tempDir().getAbsolutePath, "Table1") 10 | table.writer.append("key1", "value1".getBytes) 11 | assert("value1" == table.get("key1").get.value) 12 | } 13 | 14 | 15 | test("should write multiple values and read") { 16 | val table = new Table(TestUtils.tempDir().getAbsolutePath, "Table1") 17 | table.writer.append("key1", "value1".getBytes) 18 | table.writer.append("key2", "value2".getBytes) 19 | table.writer.append("key3", "value3".getBytes) 20 | 21 | assert("value1" == table.get("key1").get.value) 22 | assert("value2" == table.get("key2").get.value) 23 | 24 | table.writer.append("key3", "value4".getBytes) 25 | assert("value4" == table.get("key3").get.value) 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/TestUtils.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import java.io.File 4 | import java.nio.file.Paths 5 | 6 | object TestUtils { 7 | 8 | def tmpDir = { 9 | Paths.get(System.getProperty("java.io.tmpdir")) 10 | } 11 | 12 | def deleteTable(table:String) = { 13 | new File(s"${TestUtils.tmpDir}${System.getProperty("file.separator") }${table}.db").delete() 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/kvstore/VersionGeneratorTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.kvstore 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class VersionGeneratorTest extends FunSuite { 6 | test("should increment version on every call") { 7 | val versionGenerator = new VersionGenerator 8 | assert(versionGenerator.incrementAndGetVersion < versionGenerator.incrementAndGetVersion) 9 | } 10 | } 11 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/common/JsonSerDesTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.common; 2 | 3 | import org.junit.Assert; 4 | import org.junit.Test; 5 | 6 | public class JsonSerDesTest { 7 | 8 | @Test 9 | public void serializesJavaObjects() { 10 | var request = new RequestOrResponse(0, "some content", 1); 11 | String serializedMessage = JsonSerDes.serialize(request); 12 | Assert.assertEquals( 13 | "{\"requestId\":0,\"messageBodyJson\":\"some content\",\"correlationId\":1}", serializedMessage); 14 | } 15 | 16 | @Test 17 | public void deSerializesJavaObjects() { 18 | var request = new RequestOrResponse(0, "some content", 1); 19 | String serializedMessage = JsonSerDes.serialize(request); 20 | Assert.assertEquals( 21 | "{\"requestId\":0,\"messageBodyJson\":\"some content\",\"correlationId\":1}", serializedMessage); 22 | 23 | Assert.assertEquals(request, JsonSerDes.deserialize(serializedMessage, RequestOrResponse.class)); 24 | } 25 | } -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/common/TcpListenerTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.common; 2 | 3 | 4 | import org.dist.util.Networks; 5 | import org.dist.utils.JTestUtils; 6 | import org.junit.Test; 7 | 8 | import java.io.IOException; 9 | import java.net.InetAddress; 10 | 11 | import static org.junit.Assert.assertNotNull; 12 | 13 | public class TcpListenerTest { 14 | 15 | @Test 16 | public void shouldExecuteSingularUpdateQueue() throws IOException { 17 | InetAddress inetAddress = new Networks().ipv4Address(); 18 | InetAddressAndPort serverIp = InetAddressAndPort.create(inetAddress.getHostAddress(), JTestUtils.choosePort()); 19 | TcpListener tcpListener = new TcpListener(serverIp); 20 | tcpListener.start(); 21 | 22 | RequestOrResponse request = new RequestOrResponse(1, "Test String", 0); 23 | RequestOrResponse response = new Client().sendReceive(request, serverIp); 24 | 25 | assertNotNull(response); 26 | } 27 | } -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/distlog/BookieReadWriteTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog 2 | 3 | import java.nio.ByteBuffer 4 | 5 | import org.dist.queue.TestUtils 6 | import org.scalatest.FunSuite 7 | 8 | class BookieReadWriteTest extends FunSuite { 9 | 10 | test("Write and Read to Ledger") { 11 | val bookie = new Bookie(TestUtils.tempDir("bookie"), TestUtils.tempDir("ledger"), TestUtils.tempDir("cache")) 12 | val ledgerId = 1 13 | val entryId = 1 14 | val content = 1 15 | bookie.addEntry(ledgerId, createByteBuffer(content, ledgerId, entryId)); 16 | val buffer = bookie.readEntry(ledgerId, entryId) 17 | val a = buffer.getLong() 18 | val b = buffer.getLong() 19 | assert(buffer.getInt() == content) 20 | } 21 | 22 | private def createByteBuffer(i: Int, lid: Long, eid: Long) = { 23 | val bb = ByteBuffer.allocate(4 + 16) 24 | bb.putLong(lid) 25 | bb.putLong(eid) 26 | bb.putInt(i) 27 | bb.flip 28 | bb 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/distlog/LedgerCacheTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class LedgerCacheTest extends FunSuite { 6 | test("put and get offset values") { 7 | val ledgerCache = new LedgerCache(org.dist.queue.TestUtils.tempDir("bookkeeper")) 8 | ledgerCache.putEntryOffset(1, 1, 1000) 9 | ledgerCache.flush() 10 | val offset = ledgerCache.getEntryOffset(1, 1) 11 | assert(offset == 1000) 12 | } 13 | } 14 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/distlog/LedgerMetadataTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.distlog 2 | 3 | import java.util 4 | 5 | import org.dist.kvstore.InetAddressAndPort 6 | import org.dist.patterns.common.JsonSerDes 7 | import org.dist.queue.ZookeeperTestHarness 8 | import org.scalatest.FunSuite 9 | 10 | class LedgerMetadataTest extends ZookeeperTestHarness { 11 | test("should create ledger in zookeeper") { 12 | val metadata = new LedgerMetadata(3, 2) 13 | val address1 = InetAddressAndPort.create("10.10.10.10", 8000) 14 | val address2 = InetAddressAndPort.create("10.10.10.11", 8000) 15 | val address3 = InetAddressAndPort.create("10.10.10.12", 8000) 16 | metadata.addEnsemble(0, new util.ArrayList(util.Arrays.asList(address1, address2, address3))) 17 | 18 | val ledgerManager = new LedgerManager(zkClient); 19 | val path = ledgerManager.newLedgerPath(metadata) 20 | val ledgerId = ledgerManager.getLedgerId(path) 21 | assert(ledgerId == 0) 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/leaderelection/ServerTest.java: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.leaderelection; 2 | 3 | import org.junit.Test; 4 | 5 | public class ServerTest { 6 | 7 | @Test 8 | public void startup() { 9 | Server server = new Server(); 10 | server.startup(); 11 | } 12 | } -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/replicatedlog/ReplicatedKVStoreTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog 2 | 3 | import org.dist.patterns.wal.Wal 4 | import org.dist.queue.TestUtils 5 | import org.scalatest.FunSuite 6 | 7 | class ReplicatedKVStoreTest extends FunSuite { 8 | 9 | test("Should be able to compare if two logs are uptodate") { 10 | val walDir = TestUtils.tempDir("waltest") 11 | val wal = new ReplicatedWal(walDir) 12 | wal.append("test content".getBytes()) 13 | wal.append("test content2".getBytes()) 14 | wal.append("test content3".getBytes()) 15 | 16 | assert(wal.isUptoDate(3)) 17 | } 18 | 19 | test("Should be able to truncate to specific index") { 20 | val walDir = TestUtils.tempDir("waltest") 21 | val wal = new ReplicatedWal(walDir) 22 | wal.append("test content".getBytes()) 23 | wal.append("test content2".getBytes()) 24 | wal.append("test content3".getBytes()) 25 | 26 | wal.truncate(1) 27 | 28 | assert(1 == wal.readAll().size) 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/patterns/replicatedlog/heartbeat/HeartBeatSchedulerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.patterns.replicatedlog.heartbeat 2 | 3 | import org.dist.queue.TestUtils 4 | import org.scalatest.FunSuite 5 | 6 | class HeartBeatSchedulerTest extends FunSuite { 7 | 8 | test("Should invoke action when scheduler triggers") { 9 | var actionCalled = false 10 | val action = () ⇒ {actionCalled = true} 11 | val heartBeat = new HeartBeatScheduler(action) 12 | 13 | heartBeat.start() 14 | 15 | TestUtils.waitUntilTrue(()⇒{actionCalled}, "Waiting for action to be called") 16 | heartBeat.cancel() 17 | } 18 | 19 | 20 | 21 | test("Should be able to cancel scheduled task") { 22 | var actionCalled = false 23 | val action = () ⇒ {actionCalled = true} 24 | val heartBeat = new HeartBeatScheduler(action) 25 | 26 | heartBeat.start() 27 | 28 | TestUtils.waitUntilTrue(()⇒{actionCalled}, "Waiting for action to be called") 29 | 30 | actionCalled = false 31 | heartBeat.cancel() 32 | try { 33 | TestUtils.waitUntilTrue(() ⇒ { 34 | actionCalled 35 | }, "waiting for action to be called", 500) 36 | } catch { 37 | case e:RuntimeException ⇒ 38 | } 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/EmbeddedZookeeper.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue 2 | 3 | import java.net.InetSocketAddress 4 | 5 | import org.apache.zookeeper.server.{NIOServerCnxn, NIOServerCnxnFactory, ZooKeeperServer} 6 | import org.dist.queue.utils.Utils 7 | 8 | 9 | class EmbeddedZookeeper(val connectString: String) { 10 | val snapshotDir = TestUtils.tempDir() 11 | val logDir = TestUtils.tempDir() 12 | val tickTime = 500 13 | val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime) 14 | val port = connectString.split(":")(1).toInt 15 | val factory = new NIOServerCnxnFactory() 16 | factory.configure(new InetSocketAddress("127.0.0.1", port), 60) 17 | factory.startup(zookeeper) 18 | 19 | def shutdown() { 20 | factory.shutdown() 21 | Utils.rm(logDir) 22 | Utils.rm(snapshotDir) 23 | } 24 | 25 | } -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/ProducerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class ProducerTest extends FunSuite { 6 | 7 | test("should created a serialized message") { 8 | 9 | } 10 | 11 | } 12 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/TopicCreationTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue 2 | 3 | import org.dist.queue.server.{Config, Server} 4 | 5 | class TopicCreationTest extends ZookeeperTestHarness { 6 | test("should register topic and assign partitions") { 7 | val brokerId1 = 0 8 | val brokerId2 = 1 9 | 10 | val config1 = Config(brokerId1, TestUtils.hostName(), TestUtils.choosePort(), TestZKUtils.zookeeperConnect, List(TestUtils.tempDir().getAbsolutePath)) 11 | val server1 = new Server(config1) 12 | server1.startup() 13 | 14 | val config2 = Config(brokerId2, TestUtils.hostName(), TestUtils.choosePort(), TestZKUtils.zookeeperConnect, List(TestUtils.tempDir().getAbsolutePath)) 15 | val server2 = new Server(config2) 16 | server2.startup() 17 | 18 | 19 | 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/ZookeeperTestHarness.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue 2 | 3 | import akka.actor.ActorSystem 4 | import org.I0Itec.zkclient.ZkClient 5 | import org.dist.queue.utils.ZKStringSerializer 6 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, FunSuite} 7 | 8 | object TestZKUtils { 9 | val zookeeperConnect = "127.0.0.1:2182" 10 | } 11 | 12 | trait ZookeeperTestHarness extends FunSuite with BeforeAndAfterEach { 13 | val zkConnect: String = TestZKUtils.zookeeperConnect 14 | var zookeeper: EmbeddedZookeeper = null 15 | var zkClient: ZkClient = null 16 | val zkConnectionTimeout = 10000 17 | val zkSessionTimeout = 15000 18 | implicit val partitionActorSystem = ActorSystem("partitionActorSystem") 19 | 20 | override def beforeEach() = { 21 | zookeeper = new EmbeddedZookeeper(zkConnect) 22 | zkClient = new ZkClient(zookeeper.connectString, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) 23 | } 24 | 25 | override def afterEach() = { 26 | zkClient.close() 27 | zookeeper.shutdown() 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/api/LeaderAndIsrRequestTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.api 2 | 3 | import org.dist.kvstore.JsonSerDes 4 | import org.dist.queue.controller.{LeaderAndIsr, LeaderIsrAndControllerEpoch, PartitionStateInfo} 5 | import org.dist.queue.utils.ZkUtils.Broker 6 | import org.scalatest.FunSuite 7 | 8 | 9 | 10 | class LeaderAndIsrRequestTest extends FunSuite { 11 | private val topic1 = "test1" 12 | private val topic2 = "test2" 13 | private val leader1 = 0 14 | private val isr1 = List(0, 1, 2) 15 | private val leader2 = 0 16 | private val isr2 = List(0, 2, 3) 17 | 18 | test("serialize and deserialize leaderandisr request") { 19 | val leaderAndIsr1 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader1, 1, isr1, 1), 1) 20 | val leaderAndIsr2 = new LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader2, 1, isr2, 2), 1) 21 | val map: Map[(String, Int), PartitionStateInfo] = Map(("topic1", 0) → PartitionStateInfo(leaderAndIsr1, isr1.toSet), 22 | ("topic2", 0) → PartitionStateInfo(leaderAndIsr2, isr2.toSet)) 23 | val leaders = Set[Broker](Broker(1, "127.0.0.1", 8000)) 24 | val request = LeaderAndIsrRequest(map, leaders, 1, 0, 1, "client1") 25 | val str = JsonSerDes.serialize(request) 26 | val deserializedRequest = JsonSerDes.deserialize(str.getBytes, classOf[LeaderAndIsrRequest]) 27 | 28 | assert(deserializedRequest == request) 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/client/producer/DefaultPartitionerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.client.producer 2 | 3 | import org.scalatest.FunSuite 4 | 5 | class DefaultPartitionerTest extends FunSuite { 6 | 7 | test("should get random partition if for the given number of partitions") { 8 | val partitioner = new DefaultPartitioner[String]() 9 | val partitionId = partitioner.partition("FB", 3) 10 | assert(partitionId < 3) 11 | } 12 | 13 | test("should get different partition id for keys with different hashcodes") { 14 | val partitioner = new DefaultPartitioner[String]() 15 | assert(partitioner.partition("FB", 3) != partitioner.partition("ABC", 3)) 16 | } 17 | 18 | test("Gets same partition if hashcode is same") { 19 | val partitioner = new DefaultPartitioner[String]() 20 | partitioner.partition("FB", 3) == partitioner.partition("ea", 3) 21 | } 22 | 23 | } 24 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/queue/controller/ReplicaStateMachineTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.queue.controller 2 | 3 | import org.dist.queue.server.{Config, Server} 4 | import org.dist.queue.{TestUtils, TestZKUtils, ZookeeperTestHarness} 5 | 6 | class ReplicaStateMachineTest extends ZookeeperTestHarness { 7 | 8 | test("should initialize replica state for online replicas") { 9 | val brokerId1 = 0 10 | val brokerId2 = 1 11 | val brokerId3 = 2 12 | 13 | val config1 = Config(brokerId1, TestUtils.hostName(), TestUtils.choosePort(), TestZKUtils.zookeeperConnect, List(TestUtils.tempDir().getAbsolutePath)) 14 | val server1 = new Server(config1) 15 | server1.startup() 16 | 17 | val config2 = Config(brokerId2, TestUtils.hostName(), TestUtils.choosePort(), TestZKUtils.zookeeperConnect, List(TestUtils.tempDir().getAbsolutePath)) 18 | val server2 = new Server(config2) 19 | server2.startup() 20 | 21 | val config3 = Config(brokerId3, TestUtils.hostName(), TestUtils.choosePort(), TestZKUtils.zookeeperConnect, List(TestUtils.tempDir().getAbsolutePath)) 22 | val server3 = new Server(config3) 23 | server3.startup() 24 | 25 | val brokers = Map(brokerId1 → server1, brokerId2 → server2, brokerId3 → server3) 26 | 27 | 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/simplekafka/BrokerChangeListenerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.server.Config 4 | import org.dist.queue.utils.ZkUtils.Broker 5 | import org.dist.queue.{TestUtils, ZookeeperTestHarness} 6 | import org.dist.util.Networks 7 | 8 | class BrokerChangeListenerTest extends ZookeeperTestHarness { 9 | test("should add new broker information to controller on change") { 10 | val config = new Config(1, new Networks().hostname(), TestUtils.choosePort(), zkConnect, List(TestUtils.tempDir().getAbsolutePath)) 11 | val zookeeperClient: ZookeeperClientImpl = new ZookeeperClientImpl(config) 12 | zookeeperClient.registerBroker(Broker(0, "10.10.10.10", 8000)) 13 | 14 | val socketServer1 = new TestSocketServer(config) 15 | val controller = new Controller(zookeeperClient, config.brokerId, socketServer1) 16 | controller.startup() 17 | 18 | zookeeperClient.registerBroker(Broker(1, "10.10.10.11", 8001)) 19 | zookeeperClient.registerBroker(Broker(2, "10.10.10.12", 8002)) 20 | 21 | 22 | TestUtils.waitUntilTrue(() => { 23 | controller.liveBrokers.size == 3 24 | }, "Waiting for all brokers to get added", 1000) 25 | 26 | assert(controller.liveBrokers.size == 3) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/simplekafka/CreateTopicCommandZookeeperTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.{TestUtils, ZookeeperTestHarness} 4 | import org.dist.queue.server.Config 5 | import org.dist.queue.utils.ZkUtils.Broker 6 | import org.dist.util.Networks 7 | 8 | class CreateTopicCommandZookeeperTest extends ZookeeperTestHarness { 9 | test("should create persistent path for topic with topic partition assignments in zookeeper") { 10 | val config = new Config(1, new Networks().hostname(), TestUtils.choosePort(), zkConnect, List(TestUtils.tempDir().getAbsolutePath)) 11 | val zookeeperClient = new ZookeeperClientImpl(config) 12 | zookeeperClient.registerBroker(Broker(0, "10.10.10.10", 8000)) 13 | zookeeperClient.registerBroker(Broker(1, "10.10.10.11", 8001)) 14 | zookeeperClient.registerBroker(Broker(2, "10.10.10.12", 8002)) 15 | 16 | val createCommandTest = new CreateTopicCommand(zookeeperClient) 17 | createCommandTest.createTopic("topic1", 2, 3) 18 | 19 | val topics = zookeeperClient.getAllTopics() 20 | assert(topics.size == 1) 21 | 22 | val partitionAssignments = topics("topic1") 23 | assert(partitionAssignments.size == 2) 24 | partitionAssignments.foreach(p => assert(p.brokerIds.size == 3)) 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/simplekafka/PartitionConcurrentReadWriteTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import akka.actor.ActorSystem 4 | import org.dist.queue.TestUtils 5 | import org.dist.queue.common.TopicAndPartition 6 | import org.dist.queue.server.Config 7 | import org.dist.util.Networks 8 | import org.scalatest.FunSuite 9 | 10 | import scala.util.Random 11 | 12 | class PartitionConcurrentReadWriteTest extends FunSuite { 13 | implicit val partitionActorSystem = ActorSystem("partitionActorSystem") 14 | 15 | test("Concurrent write to partition should be serialized") { 16 | val config1 = Config(1, new Networks().hostname(), TestUtils.choosePort(), "", List(TestUtils.tempDir().getAbsolutePath)) 17 | 18 | val partition = new Partition(config1, TopicAndPartition("topic1", 0)) 19 | (1 to 100).foreach(i ⇒ { 20 | new Thread(() => { 21 | partition.append2(s"key${i}", s"message${i}") 22 | }).run() 23 | }) 24 | 25 | val messages: Seq[Any] = partition.read(0) 26 | assert(messages.size == 100) 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/simplekafka/ServerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.server.Config 4 | import org.dist.queue.{TestUtils, ZookeeperTestHarness} 5 | import org.dist.util.Networks 6 | import org.mockito.Mockito._ 7 | 8 | class ServerTest extends ZookeeperTestHarness { 9 | test("should register itself to zookeeper on startup") { 10 | val config = new Config(1, new Networks().hostname(), TestUtils.choosePort(), zkConnect, List(TestUtils.tempDir().getAbsolutePath)) 11 | val client: ZookeeperClient = mock(classOf[ZookeeperClient]) 12 | val leaderElector: Controller = mock(classOf[Controller]) 13 | val socketServer:SimpleSocketServer = mock(classOf[SimpleSocketServer]) 14 | var server = new Server(config, client, leaderElector, socketServer) 15 | server.startup() 16 | verify(client, atLeastOnce()).registerSelf() 17 | verify(socketServer, atLeastOnce()).startup() 18 | } 19 | 20 | test("should elect controller on startup") { 21 | val config = new Config(1, new Networks().hostname(), TestUtils.choosePort(), zkConnect, List(TestUtils.tempDir().getAbsolutePath)) 22 | val client: ZookeeperClient = mock(classOf[ZookeeperClient]) 23 | val leaderElector: Controller = mock(classOf[Controller]) 24 | val socketServer:SimpleSocketServer = mock(classOf[SimpleSocketServer]) 25 | var server = new Server(config, client, leaderElector, socketServer) 26 | server.startup() 27 | verify(client, atLeastOnce()).registerSelf() 28 | verify(leaderElector, atLeastOnce()).startup() 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/test/scala/org/dist/simplekafka/TopicChangeHandlerTest.scala: -------------------------------------------------------------------------------- 1 | package org.dist.simplekafka 2 | 3 | import org.dist.queue.server.Config 4 | import org.dist.queue.utils.ZkUtils.Broker 5 | import org.dist.queue.{TestUtils, ZookeeperTestHarness} 6 | import org.dist.util.Networks 7 | 8 | class TopicChangeHandlerTest extends ZookeeperTestHarness { 9 | 10 | class TestContext { 11 | var replicas:Seq[PartitionReplicas] = List() 12 | def leaderAndIsr(topicName:String, replicas:Seq[PartitionReplicas]) = { 13 | this.replicas = replicas 14 | } 15 | } 16 | 17 | test("should register for topic change and get replica assignments") { 18 | val config = new Config(1, new Networks().hostname(), TestUtils.choosePort(), zkConnect, List(TestUtils.tempDir().getAbsolutePath)) 19 | val zookeeperClient = new ZookeeperClientImpl(config) 20 | zookeeperClient.registerBroker(Broker(0, "10.10.10.10", 8000)) 21 | zookeeperClient.registerBroker(Broker(1, "10.10.10.11", 8001)) 22 | zookeeperClient.registerBroker(Broker(2, "10.10.10.12", 8002)) 23 | 24 | val createTopicCommand = new CreateTopicCommand(zookeeperClient) 25 | val testContext = new TestContext 26 | val topicChangeListener = new TopicChangeHandler(zookeeperClient, testContext.leaderAndIsr) 27 | zookeeperClient.subscribeTopicChangeListener(topicChangeListener) 28 | createTopicCommand.createTopic("topic1", 2, 2) 29 | 30 | TestUtils.waitUntilTrue(() => { 31 | testContext.replicas.size > 0 32 | }, "Waiting for topic metadata", 1000) 33 | } 34 | } 35 | --------------------------------------------------------------------------------