├── .gitignore ├── CONTRIBUTING.md ├── DOCUMENT.md ├── Developing.md ├── LICENSE ├── README-EN.md ├── README.md ├── core ├── pom.xml └── src │ └── main │ └── java │ └── com │ └── wuba │ └── wpaxos │ ├── Acceptor.java │ ├── AcceptorState.java │ ├── CheckpointReceiver.java │ ├── CheckpointSender.java │ ├── CommitCtx.java │ ├── CommitResult.java │ ├── Committer.java │ ├── Instance.java │ ├── IoLoop.java │ ├── Learner.java │ ├── LearnerSender.java │ ├── ProposeResult.java │ ├── Proposer.java │ ├── ProposerState.java │ ├── base │ ├── BallotNumber.java │ ├── Base.java │ ├── BaseMsg.java │ └── BroadcastMessageType.java │ ├── checkpoint │ ├── CheckpointMgr.java │ ├── Cleaner.java │ └── Replayer.java │ ├── comm │ ├── CheckpointFileInfo.java │ ├── FollowerNodeInfo.java │ ├── GroupSMInfo.java │ ├── InsideOptions.java │ ├── MasterChangeCallback.java │ ├── MembershipChangeCallback.java │ ├── MsgTransport.java │ ├── NodeInfo.java │ ├── Options.java │ ├── breakpoint │ │ ├── AcceptorBP.java │ │ ├── AlgorithmBaseBP.java │ │ ├── Breakpoint.java │ │ ├── CheckpointBP.java │ │ ├── CommiterBP.java │ │ ├── IOLoopBP.java │ │ ├── InstanceBP.java │ │ ├── LearnerBP.java │ │ ├── LogStorageBP.java │ │ ├── MasterBP.java │ │ ├── NetworkBP.java │ │ └── ProposerBP.java │ └── enums │ │ ├── CheckpointMsgType.java │ │ ├── CheckpointSendFileAckFlag.java │ │ ├── CheckpointSendFileFlag.java │ │ ├── IndexType.java │ │ ├── MessageSendType.java │ │ ├── MsgCmd.java │ │ ├── PaxosLogCleanType.java │ │ ├── PaxosMsgFlagType.java │ │ ├── PaxosMsgType.java │ │ └── TimerType.java │ ├── communicate │ ├── Communicate.java │ ├── DFNetWorker.java │ ├── NetWork.java │ ├── ReceiveMessage.java │ ├── client │ │ ├── IClient.java │ │ ├── tcp │ │ │ ├── TcpClient.java │ │ │ └── TcpClientHander.java │ │ └── udp │ │ │ └── UDPClient.java │ ├── config │ │ ├── ClientConfig.java │ │ └── ServerConfig.java │ └── server │ │ ├── IServer.java │ │ ├── tcp │ │ ├── TcpHandler.java │ │ ├── TcpPipelineFactory.java │ │ └── TcpServer.java │ │ └── udp │ │ ├── UdpHandler.java │ │ ├── UdpPipelineFactory.java │ │ └── UdpServer.java │ ├── config │ ├── Config.java │ ├── Def.java │ ├── PaxosNodeFunctionRet.java │ ├── PaxosTryCommitRet.java │ ├── WriteOptions.java │ └── WriteState.java │ ├── exception │ └── SerializeException.java │ ├── helper │ ├── LearnerState.java │ ├── MsgCounter.java │ ├── SerialLock.java │ └── WaitLock.java │ ├── master │ ├── MasterInfo.java │ ├── MasterMgr.java │ ├── MasterStateMachine.java │ └── MasterVariablesStore.java │ ├── node │ ├── Group.java │ ├── Node.java │ ├── PNode.java │ └── ProposeBatch.java │ ├── proto │ ├── AcceptorStateData.java │ ├── BatchPaxosValue.java │ ├── CheckpointMsg.java │ ├── Header.java │ ├── MasterOperator.java │ ├── MasterVariables.java │ ├── PaxosMsg.java │ ├── PaxosNodeInfo.java │ ├── PaxosValue.java │ ├── Proto.java │ ├── ProtocolConst.java │ ├── SystemVariables.java │ └── pb │ │ ├── MasterSmProto.java │ │ ├── PaxosProto.java │ │ ├── masterSmProto.proto │ │ └── paxosProto.proto │ ├── store │ ├── AppendDataCallback.java │ ├── AppendDataResult.java │ ├── AppendDataStatus.java │ ├── DataBase.java │ ├── DefaultDataBase.java │ ├── DefaultLogStorage.java │ ├── FileID.java │ ├── GetResult.java │ ├── LogStorage.java │ ├── PaxosLog.java │ ├── PhysicLog.java │ ├── PutDataResult.java │ ├── PutDataStatus.java │ ├── StoreCheckpoint.java │ ├── StoreStatus.java │ ├── SystemVariablesStore.java │ ├── TransientStorePool.java │ ├── VarStorage.java │ ├── config │ │ ├── DynamicConfig.java │ │ ├── FlushDiskType.java │ │ ├── StoreConfig.java │ │ ├── StoreConfigLoader.java │ │ └── StorePathConfigHelper.java │ ├── db │ │ ├── FileIndexDB.java │ │ ├── IndexDB.java │ │ └── LevelDBIndex.java │ ├── pagecache │ │ ├── MapedFile.java │ │ ├── MapedFileQueue.java │ │ └── ReferenceResource.java │ └── service │ │ ├── AllocateMapedFileService.java │ │ ├── CleanIndexDBService.java │ │ ├── CleanIndexMappedFileService.java │ │ ├── CleanPhyMappedFileService.java │ │ ├── CleanPhysicLogService.java │ │ ├── CommitIndexService.java │ │ ├── CommitPhysicLogService.java │ │ ├── FlushIndexService.java │ │ └── FlushPhysicLogService.java │ ├── storemachine │ ├── BatchSMCtx.java │ ├── InsideSM.java │ ├── SMCtx.java │ ├── SMFac.java │ ├── StateMachine.java │ └── SystemVSM.java │ └── utils │ ├── ByteConverter.java │ ├── ConfigManager.java │ ├── CountDownLatch2.java │ ├── Crc32.java │ ├── FileInfo.java │ ├── FileUtils.java │ ├── JavaOriTypeWrapper.java │ ├── LibC.java │ ├── MasterGroupStatPrinter.java │ ├── MixAll.java │ ├── Notifier.java │ ├── NotifierPool.java │ ├── OtherUtils.java │ ├── RemotingUtil.java │ ├── ServiceThread.java │ ├── ThreadFactoryImpl.java │ ├── ThreadRenameFactory.java │ ├── Time.java │ ├── TimeStat.java │ ├── Timer.java │ └── UtilAll.java ├── img ├── clean.png ├── framework.png ├── store.png ├── structure.png └── wpaxos-wechat.png ├── pom.xml └── sample ├── pom.xml └── src └── main └── java └── com └── wuba └── wpaxos └── sample ├── echo ├── EchoClient.java ├── EchoSM.java ├── EchoSMCtx.java ├── EchoServer.java └── start.sh ├── election ├── ElectionMain.java ├── ElectionServer.java └── start.sh ├── kv ├── CheckpointManager.java ├── ClientRole.java ├── KVClientLeaner.java ├── KVClientMaster.java ├── KVClientSlave.java ├── KVOPValue.java ├── KVOperation.java ├── KVResult.java ├── KVTestResult.java ├── KVTestSM.java ├── KVTestServer.java ├── rocksdb │ ├── RocksDB.java │ ├── RocksDBHolder.java │ └── RocksDbConfig.java └── start.sh ├── simple ├── SimpleClient.java ├── SimpleSM.java ├── SimpleServer.java └── start.sh └── util └── NodeUtil.java /CONTRIBUTING.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/CONTRIBUTING.md -------------------------------------------------------------------------------- /DOCUMENT.md: -------------------------------------------------------------------------------- 1 | # WPaxos项目结构 2 | 3 | 4 | ## PNode 5 | 6 | WPaxos服务集群包括多个独立运行的Node物理节点,每个Node节点有固定数量的Paxos Group,每个group下运行一个Paxos实例,多个Paxos Group可以同时确定多组instanceID递增的有序序列,Paxos实例每确定一个值并提交至状态机执行成功,则instanceID增加1,一旦一个instanceID对应的值被确定,之后不会被修改。 7 | ## Logstorage 8 | 存储模块业务可自定义实现。为了保证任何情况下已提交的instance数据不丢,Node节点日志存储PaxosLog需要实时持久化存储Acceptor已接受的所有instance数据,并且每条instance数据需支持随机读取。为此,WPaxos默认PaxosLog存储由物理文件和IndexDB索引两部分构成,如下图所示,IndexDB默认采用LevelDB实现,physiclog采用文件顺序追加写+同步刷盘方式,IndexDB则采用异步刷盘,当进程异常挂掉时,只要保证physiclog数据不丢,即可通过PaxosLog重放,恢复IndexDB中的索引数。 9 | 10 | 但在实际测试中发现,LevelDB异步写接口时常会发生几百毫秒的存储抖动,为了解决这个问题,在Paxos group数量比较少时,我们将IndexDB实现改为了内存缓存+异步写文件方式,解决了抖动问题,同时便于历史日志以文件为单位批量清理。 11 | WPaxos同时支持按照instance保留数量(HoldCount)、保留时间(HoldTime)以及磁盘空间占用率三种方式清理PaxosLog历史日志。 12 | 13 | ## StateMachine 14 | 一个Paxos实例可以挂载多种状态机,其中Master StateMachine与System StateMachine为服务内置状态机,前者用来实现Master选举、续约状态管理,后者用来实现Member Ship成员动态变更管理。用户可以将核心业务逻辑添加到自定义状态机execute方法中执行。一个Paxos实例中任何节点,在执行同一条instance数据的时候,看到的状态机所有数据状态,都是一致的。 15 | ## Checkpoint 16 | Checkpoint为状态机数据快照,状态机可通过execute方法执行实时更新快照,也可在checkpoint replayer中重读数据异步生成,通过快照可恢复状态机数据,但Checkpoint同步是个比较重的过程,需要删除所有历史数据来重构状态机。在Paxos实例中任何节点都可以作为Learner,定时向其它节点询问自己数据是否有落后,若某节点最大instanceID小于一定时间内返回Ack的所有节点最小chosen instanceID,则选择同步checkpoint来对齐数据。 17 | ## Network 18 | 同样,网络通信模块用户也可自定义。WPaxos默认实现中,可指定创建多少组网络IO实例,每组实例中会初始化一个本节点与Paxos实例中其它所有节点的TCP连接以及UDPClient。在一个Paxos实例中,来自同一个节点的所有请求都是串行的,因此默认初始化IO实例数为Paxos Group数量,即可满足性能最大化需求。 19 | -------------------------------------------------------------------------------- /Developing.md: -------------------------------------------------------------------------------- 1 | ## Developers 2 | 3 | |Github ID|Organization|TimeZone| 4 | |:-:|:-:|:-:| 5 | |liuliuwd|58.com|+8| 6 | |Begro|58.com|+8| 7 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/CommitResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos; 17 | 18 | /** 19 | * commit result 20 | */ 21 | public class CommitResult { 22 | int commitRet; 23 | long succInstanceID; 24 | 25 | public CommitResult(int commitRet, long succInstanceID) { 26 | super(); 27 | this.commitRet = commitRet; 28 | this.succInstanceID = succInstanceID; 29 | } 30 | 31 | public int getCommitRet() { 32 | return commitRet; 33 | } 34 | 35 | public void setCommitRet(int commitRet) { 36 | this.commitRet = commitRet; 37 | } 38 | 39 | public long getSuccInstanceID() { 40 | return succInstanceID; 41 | } 42 | 43 | public void setSuccInstanceID(long succInstanceID) { 44 | this.succInstanceID = succInstanceID; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/ProposeResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos; 17 | 18 | /** 19 | * propose result 20 | */ 21 | public class ProposeResult { 22 | // ret, 0 if success 23 | private int result; 24 | // return when propose success 25 | private long instanceID = 0; 26 | 27 | public ProposeResult(int result, long instanceID) { 28 | super(); 29 | this.result = result; 30 | this.instanceID = instanceID; 31 | } 32 | 33 | public ProposeResult(int result) { 34 | super(); 35 | this.result = result; 36 | } 37 | 38 | public int getResult() { 39 | return result; 40 | } 41 | 42 | public void setResult(int result) { 43 | this.result = result; 44 | } 45 | 46 | public long getInstanceID() { 47 | return instanceID; 48 | } 49 | 50 | public void setInstanceID(long instanceID) { 51 | this.instanceID = instanceID; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/ProposerState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.base.BallotNumber; 22 | import com.wuba.wpaxos.config.Config; 23 | 24 | /** 25 | * proposer current state 26 | */ 27 | public class ProposerState { 28 | private static final Logger logger = LogManager.getLogger(ProposerState.class); 29 | private long proposalID; 30 | private long highestOtherProposalID; 31 | private byte[] value; 32 | private BallotNumber highestOtherPreAcceptBallot = new BallotNumber(0, 0); 33 | private Config config; 34 | 35 | public ProposerState(Config config) { 36 | this.config = config; 37 | this.proposalID = 1; 38 | init(); 39 | } 40 | 41 | public void init() { 42 | this.highestOtherProposalID = 0; 43 | this.value = new byte[]{}; 44 | } 45 | 46 | public void setStartProposalID(long proposalID) { 47 | this.proposalID = proposalID; 48 | } 49 | 50 | public void newPrepare() { 51 | logger.info("START ProposalID {} HighestOther {} MyNodeID {}.", this.proposalID, this.highestOtherProposalID, this.config.getMyNodeID()); 52 | 53 | long maxProposalId = this.proposalID > this.highestOtherProposalID ? this.proposalID : this.highestOtherProposalID; 54 | this.proposalID = maxProposalId + 1; 55 | } 56 | 57 | public void addPreAcceptValue(BallotNumber otherPreAcceptBallot, byte[] otherPreAcceptValue) { 58 | if(otherPreAcceptBallot.isNull()) { 59 | return ; 60 | } 61 | 62 | if (otherPreAcceptBallot.gt(this.highestOtherPreAcceptBallot)) { 63 | this.highestOtherPreAcceptBallot = otherPreAcceptBallot; 64 | this.value = otherPreAcceptValue; 65 | } 66 | } 67 | 68 | public long getProposalID() { 69 | return proposalID; 70 | } 71 | 72 | public byte[] getValue() { 73 | return this.value; 74 | } 75 | 76 | public void setValue(byte[] value) { 77 | this.value = value; 78 | } 79 | 80 | public void setOtherProposalID(long otherProposalID) { 81 | if(otherProposalID > this.highestOtherProposalID) { 82 | this.highestOtherProposalID = otherProposalID; 83 | } 84 | } 85 | 86 | public void resetHighestOtherPreAcceptBallot() { 87 | this.highestOtherPreAcceptBallot.reset(); 88 | } 89 | } 90 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/base/BallotNumber.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.base; 17 | 18 | /** 19 | * global unique proposalID 20 | */ 21 | public class BallotNumber { 22 | private long proposalID = 0; 23 | private long nodeId = 0; 24 | 25 | public BallotNumber(long proposalID, long nodeId) { 26 | super(); 27 | this.proposalID = proposalID; 28 | this.nodeId = nodeId; 29 | } 30 | 31 | public boolean ge(BallotNumber other) { 32 | if (this.proposalID == other.getProposalID()) { 33 | return this.nodeId >= other.getNodeId(); 34 | } else { 35 | return this.proposalID >= other.getProposalID(); 36 | } 37 | } 38 | 39 | public boolean gt(BallotNumber other) { 40 | if (this.proposalID == other.getProposalID()) { 41 | return this.nodeId > other.getNodeId(); 42 | } else { 43 | return this.proposalID > other.getProposalID(); 44 | } 45 | } 46 | 47 | public void reset() { 48 | this.proposalID = 0; 49 | this.nodeId = 0; 50 | } 51 | 52 | public boolean isNull() { 53 | return this.proposalID == 0; 54 | } 55 | 56 | public boolean unequals(Object obj) { 57 | return !equals(obj); 58 | } 59 | 60 | @Override 61 | public boolean equals(Object obj) { 62 | if (this == obj) 63 | return true; 64 | if (obj == null) 65 | return false; 66 | if (getClass() != obj.getClass()) 67 | return false; 68 | BallotNumber other = (BallotNumber) obj; 69 | if (nodeId != other.nodeId) 70 | return false; 71 | if (proposalID != other.proposalID) 72 | return false; 73 | return true; 74 | } 75 | 76 | public long getProposalID() { 77 | return proposalID; 78 | } 79 | 80 | public void setProposalID(long proposalID) { 81 | this.proposalID = proposalID; 82 | } 83 | 84 | public long getNodeId() { 85 | return nodeId; 86 | } 87 | 88 | public void setNodeId(long nodeId) { 89 | this.nodeId = nodeId; 90 | } 91 | 92 | @Override 93 | public int hashCode() { 94 | final int prime = 31; 95 | int result = 1; 96 | result = prime * result + (int) (nodeId ^ (nodeId >>> 32)); 97 | result = prime * result + (int) (proposalID ^ (proposalID >>> 32)); 98 | return result; 99 | } 100 | 101 | @Override 102 | public String toString() { 103 | return "BallotNumber [proposalID=" + proposalID + ", nodeId=" + nodeId + "]"; 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/base/BroadcastMessageType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.base; 17 | 18 | /** 19 | * 消息广播通知类型 20 | */ 21 | public enum BroadcastMessageType { 22 | BroadcastMessage_Type_RunSelf_First(1), 23 | BroadcastMessage_Type_RunSelf_Final(2), 24 | BroadcastMessage_Type_RunSelf_None(3); 25 | 26 | int type; 27 | 28 | private BroadcastMessageType(int type) { 29 | this.type = type; 30 | } 31 | 32 | public int getType() { 33 | return type; 34 | } 35 | 36 | public void setType(int type) { 37 | this.type = type; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/CheckpointFileInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | /** 19 | * checkpoint文件相关信息 20 | */ 21 | public class CheckpointFileInfo { 22 | private String sFilePath; 23 | private int fileSize; 24 | 25 | public CheckpointFileInfo(String sFilePath, int fileSize) { 26 | super(); 27 | this.sFilePath = sFilePath; 28 | this.fileSize = fileSize; 29 | } 30 | 31 | public String getsFilePath() { 32 | return sFilePath; 33 | } 34 | 35 | public void setsFilePath(String sFilePath) { 36 | this.sFilePath = sFilePath; 37 | } 38 | 39 | public int getFileSize() { 40 | return fileSize; 41 | } 42 | 43 | public void setFileSize(int fileSize) { 44 | this.fileSize = fileSize; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/FollowerNodeInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | /** 19 | * follower节点信息 20 | */ 21 | public class FollowerNodeInfo { 22 | private NodeInfo myNode; 23 | private NodeInfo followNode; 24 | 25 | public NodeInfo getMyNode() { 26 | return myNode; 27 | } 28 | 29 | public void setMyNode(NodeInfo myNode) { 30 | this.myNode = myNode; 31 | } 32 | 33 | public NodeInfo getFollowNode() { 34 | return followNode; 35 | } 36 | 37 | public void setFollowNode(NodeInfo followNode) { 38 | this.followNode = followNode; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/GroupSMInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | import java.util.LinkedList; 19 | import java.util.List; 20 | 21 | import com.wuba.wpaxos.storemachine.StateMachine; 22 | 23 | /** 24 | * 每个group状态机相关信息 25 | */ 26 | public class GroupSMInfo { 27 | private int groupIdx; 28 | private List smList = new LinkedList(); 29 | private boolean isUseMaster; 30 | 31 | public GroupSMInfo() {} 32 | 33 | public void addSM(StateMachine sm) { 34 | this.smList.add(sm); 35 | } 36 | 37 | public int getGroupIdx() { 38 | return groupIdx; 39 | } 40 | 41 | public void setGroupIdx(int groupIdx) { 42 | this.groupIdx = groupIdx; 43 | } 44 | 45 | public List getSmList() { 46 | return smList; 47 | } 48 | 49 | public void setSmList(List smList) { 50 | this.smList = smList; 51 | } 52 | 53 | public boolean isUseMaster() { 54 | return isUseMaster; 55 | } 56 | 57 | public void setUseMaster(boolean isUseMaster) { 58 | this.isUseMaster = isUseMaster; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/MasterChangeCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | /** 19 | * master变更回调 20 | */ 21 | public interface MasterChangeCallback { 22 | public void callback(int groupIdx, boolean masterChange,boolean isMaster, boolean isCheckpoint); 23 | } 24 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/MembershipChangeCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | import java.util.List; 19 | 20 | /** 21 | * 成员变更回调 22 | */ 23 | public interface MembershipChangeCallback { 24 | public void callback(int groupIdx, List nodeInfoList); 25 | } 26 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/MsgTransport.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | /** 19 | * 数据发送接口 20 | */ 21 | public interface MsgTransport { 22 | 23 | // Message_SendType_UDP 24 | public int sendMessage(int groupIdx, long sendtoNodeID, byte[] sBuffer, int sendType); 25 | 26 | // Message_SendType_UDP 27 | public int broadcastMessage(int groupIdx, byte[] sBuffer, int sendType); 28 | 29 | // Message_SendType_UDP 30 | public int broadcastMessageFollower(int groupIdx, byte[] sBuffer, int sendType); 31 | 32 | // Message_SendType_UDP 33 | public int broadcastMessageTempNode(int groupIdx, byte[] sBuffer, int sendType); 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/NodeInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.utils.ByteConverter; 22 | 23 | import java.util.Objects; 24 | 25 | /** 26 | * NodeInfo, paxos group member 27 | */ 28 | public class NodeInfo { 29 | private static final Logger logger = LogManager.getLogger(NodeInfo.class); 30 | private long nodeID = 0; 31 | private String ip = ""; 32 | private int port = -1; 33 | 34 | public NodeInfo() { 35 | } 36 | 37 | public NodeInfo(long nodeID) { 38 | this.nodeID = nodeID; 39 | parseNodeID(); 40 | } 41 | 42 | public NodeInfo(String ip, int port) throws Exception { 43 | this.ip = ip; 44 | this.port = port; 45 | makeNodeID(); 46 | } 47 | 48 | public long getNodeID() { 49 | return nodeID; 50 | } 51 | 52 | public void setNodeID(long nodeID) { 53 | this.nodeID = nodeID; 54 | } 55 | 56 | public String getIp() { 57 | return ip; 58 | } 59 | 60 | public void setIp(String ip) { 61 | this.ip = ip; 62 | try { 63 | makeNodeID(); 64 | } catch (Exception e) { 65 | logger.error(e.getMessage(), e); 66 | } 67 | } 68 | 69 | public int getPort() { 70 | return port; 71 | } 72 | 73 | public void setPort(int port) { 74 | this.port = port; 75 | try { 76 | makeNodeID(); 77 | } catch (Exception e) { 78 | logger.error(e.getMessage(), e); 79 | } 80 | } 81 | 82 | public void setIpPort(String ip, int port) { 83 | this.ip = ip; 84 | this.port = port; 85 | try { 86 | makeNodeID(); 87 | } catch (Exception e) { 88 | logger.error(e.getMessage(), e); 89 | } 90 | } 91 | 92 | public String getIpPort() { 93 | String addr = this.ip + ":" + this.port; 94 | return addr; 95 | } 96 | 97 | private void makeNodeID() throws Exception { 98 | int ipInt = ByteConverter.ipToInt(ip); 99 | nodeID = ((long)ipInt) << 32 | port; 100 | } 101 | 102 | private void parseNodeID() { 103 | port = (int) (nodeID & (0xffffffff)); 104 | int ipInt = (int) (nodeID >> 32); 105 | ip = ByteConverter.getIpStr(ipInt); 106 | } 107 | 108 | public static long parseNodeID(String sip, int iport) throws Exception { 109 | int ipInt = ByteConverter.ipToInt(sip); 110 | long lnodeID = ((long)ipInt) << 32 | iport; 111 | return lnodeID; 112 | } 113 | 114 | @Override 115 | public boolean equals(Object o) { 116 | if (this == o) return true; 117 | if (o == null || getClass() != o.getClass()) return false; 118 | NodeInfo nodeInfo = (NodeInfo) o; 119 | return nodeID == nodeInfo.nodeID && 120 | port == nodeInfo.port && 121 | Objects.equals(ip, nodeInfo.ip); 122 | } 123 | 124 | @Override 125 | public int hashCode() { 126 | return Objects.hash(nodeID, ip, port); 127 | } 128 | 129 | @Override 130 | public String toString() { 131 | return "NodeInfo [nodeID=" + nodeID + ", ip=" + ip + ", port=" + port + "]"; 132 | } 133 | 134 | } 135 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/AcceptorBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * Acceptor执行断点跟踪 23 | */ 24 | public class AcceptorBP { 25 | private static final Logger logger = LogManager.getLogger(AcceptorBP.class); 26 | 27 | public void onPrepare(int groupId, long instanceID) { 28 | logger.debug("AcceptorBP onPrepare, group : {}, instanceID {}.", groupId, instanceID); 29 | } 30 | 31 | public void onPreparePass(int groupId, long instanceID) { 32 | logger.debug("AcceptorBP onPreparePass, group : {}, instanceID {}.", groupId, instanceID); 33 | } 34 | 35 | public void onPreparePersistFail(int groupId, long instanceID) { 36 | logger.debug("AcceptorBP onPreparePersistFail, group : {}, instanceID {}.", groupId, instanceID); 37 | } 38 | 39 | public void onPrepareReject(int groupId, long instanceID) { 40 | logger.debug("AcceptorBP onPrepareReject, group : {}, instanceID {}.", groupId, instanceID); 41 | } 42 | 43 | public void onAccept(int groupId, long instanceID) { 44 | logger.debug("AcceptorBP onAccept, group : {}, instanceID {}.", groupId, instanceID); 45 | } 46 | 47 | public void onAcceptPass(int groupId, long instanceID) { 48 | logger.debug("AcceptorBP onAcceptPass, group : {}, instanceID {}.", groupId, instanceID); 49 | } 50 | 51 | public void onAcceptPersistFail(int groupId, long instanceID) { 52 | logger.debug("AcceptorBP onAcceptPersistFail, group : {}, instanceID {}.", groupId, instanceID); 53 | } 54 | 55 | public void onAcceptReject(int groupId, long instanceID) { 56 | logger.info("AcceptorBP onAcceptReject, group : {}, instanceID {}.", groupId, instanceID); 57 | } 58 | 59 | } 60 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/AlgorithmBaseBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * 协议序列化断点跟踪 23 | */ 24 | public class AlgorithmBaseBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(AlgorithmBaseBP.class); 27 | 28 | public void unPackHeaderLenTooLong() { 29 | logger.debug("AlgorithmBaseBP unPackHeaderLenTooLong"); 30 | } 31 | 32 | public void unPackChecksumNotSame() { 33 | logger.debug("AlgorithmBaseBP unPackChecksumNotSame"); 34 | } 35 | 36 | public void headerGidNotSame() { 37 | logger.debug("AlgorithmBaseBP headerGidNotSame"); 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/Breakpoint.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | /** 19 | * 断点执行日志跟踪 20 | */ 21 | public class Breakpoint { 22 | 23 | private ProposerBP proposerBP = new ProposerBP(); 24 | 25 | private AcceptorBP acceptorBP = new AcceptorBP(); 26 | 27 | private LearnerBP learnerBP = new LearnerBP(); 28 | 29 | private InstanceBP instanceBP = new InstanceBP(); 30 | 31 | private CommiterBP commiterBP = new CommiterBP(); 32 | 33 | private IOLoopBP ioLoopBP = new IOLoopBP(); 34 | 35 | private NetworkBP networkBP = new NetworkBP(); 36 | 37 | private LogStorageBP logStorageBP = new LogStorageBP(); 38 | 39 | private AlgorithmBaseBP algorithmBaseBP = new AlgorithmBaseBP(); 40 | 41 | private CheckpointBP checkpointBP = new CheckpointBP(); 42 | 43 | private MasterBP masterBP = new MasterBP(); 44 | 45 | private static Breakpoint breakpoint = new Breakpoint(); 46 | 47 | private static boolean isLogOpen = true; 48 | 49 | private Breakpoint() { 50 | } 51 | 52 | public static Breakpoint getInstance() { 53 | return breakpoint; 54 | } 55 | 56 | public void setInstance(Breakpoint newBreakpoint) { 57 | breakpoint = newBreakpoint; 58 | } 59 | 60 | public ProposerBP getProposerBP() { 61 | return this.proposerBP; 62 | } 63 | 64 | public AcceptorBP getAcceptorBP() { 65 | return this.acceptorBP; 66 | } 67 | 68 | public LearnerBP getLearnerBP() { 69 | return this.learnerBP; 70 | } 71 | 72 | public InstanceBP getInstanceBP() { 73 | return this.instanceBP; 74 | } 75 | 76 | public CommiterBP getCommiterBP() { 77 | return this.commiterBP; 78 | } 79 | 80 | public IOLoopBP getIOLoopBP() { 81 | return this.ioLoopBP; 82 | } 83 | 84 | public NetworkBP getNetworkBP() { 85 | return this.networkBP; 86 | } 87 | 88 | public LogStorageBP getLogStorageBP() { 89 | return this.logStorageBP; 90 | } 91 | 92 | public AlgorithmBaseBP getAlgorithmBaseBP() { 93 | return this.algorithmBaseBP; 94 | } 95 | 96 | public CheckpointBP getCheckpointBP() { 97 | return this.checkpointBP; 98 | } 99 | 100 | public MasterBP getMasterBP() { 101 | return this.masterBP; 102 | } 103 | 104 | public void setInstanceBP(InstanceBP instanceBP) { 105 | this.instanceBP = instanceBP; 106 | } 107 | 108 | public static boolean isLogOpen() { 109 | return isLogOpen; 110 | } 111 | 112 | public static void setLogOpen(boolean isLogOpen) { 113 | Breakpoint.isLogOpen = isLogOpen; 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/CheckpointBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * checkpoint执行断点跟踪 23 | */ 24 | public class CheckpointBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(CheckpointBP.class); 27 | 28 | public void needAskforCheckpoint() { 29 | logger.debug("CheckpointBP needAskforCheckpoint"); 30 | } 31 | 32 | public void sendCheckpointOneBlock() { 33 | logger.debug("CheckpointBP sendCheckpointOneBlock"); 34 | } 35 | 36 | public void onSendCheckpointOneBlock() { 37 | logger.debug("CheckpointBP onSendCheckpointOneBlock"); 38 | } 39 | 40 | public void sendCheckpointBegin() { 41 | logger.debug("CheckpointBP sendCheckpointBegin"); 42 | } 43 | 44 | public void sendCheckpointEnd() { 45 | logger.debug("CheckpointBP sendCheckpointEnd"); 46 | } 47 | 48 | public void receiveCheckpointDone() { 49 | logger.debug("CheckpointBP receiveCheckpointDone"); 50 | } 51 | 52 | public void receiveCheckpointAndLoadFail() { 53 | logger.debug("CheckpointBP receiveCheckpointAndLoadFail"); 54 | } 55 | 56 | public void receiveCheckpointAndLoadSucc() { 57 | logger.debug("CheckpointBP receiveCheckpointAndLoadSucc"); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/CommiterBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * commiter断点执行日志跟踪 23 | */ 24 | public class CommiterBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(CommiterBP.class); 27 | 28 | public void newValue(int groupId) { 29 | logger.debug("CommiterBP newValue, group : {}.", groupId); 30 | } 31 | 32 | public void newValueConflict(int groupId, long instanceID) { 33 | logger.debug("CommiterBP newValueConflict, group : {}, instanceID {}.", groupId, instanceID); 34 | } 35 | 36 | public void newValueGetLockTimeout(int groupId) { 37 | logger.debug("CommiterBP newValueGetLockTimeout, group : {}.", groupId); 38 | } 39 | 40 | public void newValueGetLockReject(int groupId) { 41 | logger.debug("CommiterBP newValueGetLockReject, group : {}.", groupId); 42 | } 43 | 44 | public void newValueGetLockOK(int useTimeMs, int groupId, long instanceID) { 45 | logger.debug("CommiterBP newValueGetLockOK useTimeMs : {}, groupId : {}, instanceID :{}.",useTimeMs, groupId,instanceID); 46 | } 47 | 48 | public void newValueCommitOK(int useTimeMs, int groupId) { 49 | logger.debug("CommiterBP newValueCommitOK useTimeMs {}, group {}.",useTimeMs, groupId); 50 | } 51 | 52 | public void newValueCommitFail(int groupId, long instanceID) { 53 | logger.debug("CommiterBP newValueCommitFail, group : {}, instanceID :{}.", groupId, instanceID); 54 | } 55 | 56 | public void batchPropose(int groupId) { 57 | logger.debug("CommiterBP batchPropose, group : {}.", groupId); 58 | } 59 | 60 | public void batchProposeOK(int groupId) { 61 | logger.debug("CommiterBP batchProposeOK, group : {}.", groupId); 62 | } 63 | 64 | public void batchProposeFail(int groupId) { 65 | logger.debug("CommiterBP batchProposeFail, group : {}.", groupId); 66 | } 67 | 68 | public void batchProposeWaitTimeMs(int waitTimeMs, int groupId) { 69 | logger.debug("CommiterBP batchProposeWaitTimeMs waitTimeMs {} , groupId : {}.", waitTimeMs, groupId); 70 | } 71 | 72 | public void batchProposeDoPropose(int batchCount, int groupId) { 73 | logger.debug("CommiterBP batchProposeDoPropose batchCount {}, groupId : {}.", batchCount, groupId); 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/IOLoopBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * IOLoop断点日志跟踪 23 | */ 24 | public class IOLoopBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(IOLoopBP.class); 27 | 28 | public void oneLoop(int groupId) { 29 | logger.debug("IOLoopBP oneLoop, group : {}.", groupId); 30 | } 31 | 32 | public void enqueueMsg(int groupId) { 33 | logger.debug("IOLoopBP enqueueMsg, group : {}.", groupId); 34 | } 35 | 36 | public void enqueueMsgRejectByFullQueue(int groupId) { 37 | logger.debug("IOLoopBP enqueueMsgRejectByFullQueue, group : {}.", groupId); 38 | } 39 | 40 | public void enqueueRetryMsg(int groupId) { 41 | logger.debug("IOLoopBP enqueueRetryMsg, group : {}.", groupId); 42 | } 43 | 44 | public void enqueueRetryMsgRejectByFullQueue(int groupId) { 45 | logger.debug("IOLoopBP enqueueRetryMsgRejectByFullQueue, group : {}.", groupId); 46 | } 47 | 48 | public void outQueueMsg(int groupId) { 49 | logger.debug("IOLoopBP outQueueMsg, group : {}.", groupId); 50 | } 51 | 52 | public void dealWithRetryMsg(int groupId) { 53 | logger.debug("IOLoopBP dealWithRetryMsg, group : {}.", groupId); 54 | } 55 | 56 | } 57 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/LogStorageBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * 数据存储断点日志跟踪 23 | */ 24 | public class LogStorageBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(LogStorageBP.class); 27 | 28 | public void fileIDToValueFail(int groupId, long instanceID) { 29 | logger.debug("LogStorageBP fileIDToValueFail, group : {}, instanceID {}.", groupId, instanceID); 30 | } 31 | 32 | public void valueToFileIDFail(int groupId, long instanceID) { 33 | logger.debug("LogStorageBP valueToFileIDFail, group : {}, instanceID {}.", groupId, instanceID); 34 | } 35 | 36 | public void indexDBPutFail(int groupId, long instanceID) { 37 | logger.debug("LogStorageBP indexDBPutFail, group : {}, instanceID {}.", groupId, instanceID); 38 | } 39 | 40 | public void indexDBPutOK(int useTimeMs, int groupId, long instanceID) { 41 | logger.debug("LogStorageBP indexDBPutOK useTimeMs : {}, group : {}, instanceID {}.", useTimeMs, groupId, instanceID); 42 | } 43 | 44 | public void appendDataFail(int groupId, long instanceID) { 45 | logger.debug("LogStorageBP appendDataFail, group : {}, instanceID {}.", groupId, instanceID); 46 | } 47 | 48 | public void appendDataOK(int writeLen, int useTimeMs, int groupId, long instanceID) { 49 | logger.debug("LogStorageBP appendDataOK writeLen : {}, useTimeMs : {}, group : {}, instanceID {}.", writeLen, useTimeMs, groupId, instanceID); 50 | } 51 | 52 | public void getFileChecksumNotEquel(int groupId, long instanceID) { 53 | logger.debug("LogStorageBP getFileChecksumNotEquel, group : {}, instanceID {}.", groupId, instanceID); 54 | } 55 | 56 | } 57 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/MasterBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * master相关日志跟踪 23 | */ 24 | public class MasterBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(MasterBP.class); 27 | 28 | public void tryBeMaster(int groupId) { 29 | logger.debug("MasterBP tryBeMaster, group : {}.", groupId); 30 | } 31 | 32 | public void tryBeMasterProposeFail(int groupId) { 33 | logger.debug("MasterBP tryBeMasterProposeFail, group : {},", groupId); 34 | } 35 | 36 | public void successBeMaster(int groupId) { 37 | logger.debug("MasterBP successBeMaster, group : {}.", groupId); 38 | } 39 | 40 | public void otherBeMaster(int groupId) { 41 | logger.debug("MasterBP otherBeMaster, group : {}.", groupId); 42 | } 43 | 44 | public void dropMaster(int groupId) { 45 | logger.debug("MasterBP dropMaster, group : {}.", groupId); 46 | } 47 | 48 | public void toBeMaster(int groupId) { 49 | logger.debug("MasterBP toBeMaster, group : {}.", groupId); 50 | } 51 | 52 | public void toBeMasterFail(int groupId) { 53 | logger.debug("MasterBP toBeMaster Fail, group : {}.", groupId); 54 | } 55 | 56 | public void masterSMInconsistent(int groupId) { 57 | logger.debug("MasterBP masterSMInconsistent, group : {}.", groupId); 58 | } 59 | 60 | } 61 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/breakpoint/NetworkBP.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.breakpoint; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | /** 22 | * 网络模块断点日志跟踪 23 | */ 24 | public class NetworkBP { 25 | 26 | private static final Logger logger = LogManager.getLogger(NetworkBP.class); 27 | 28 | public void tcpEpollLoop(int groupId) { 29 | logger.debug("NetworkBP tcpEpollLoop, group : {}.", groupId); 30 | } 31 | 32 | public void tcpOnError(int groupId) { 33 | logger.debug("NetworkBP tcpOnError, group : {}.", groupId); 34 | } 35 | 36 | public void tcpAcceptFd(int groupId) { 37 | logger.debug("NetworkBP tcpAcceptFd, group : {}.", groupId); 38 | } 39 | 40 | public void tcpQueueFull(int groupId) { 41 | logger.debug("NetworkBP tcpQueueFull, group : {}.", groupId); 42 | } 43 | 44 | public void tcpReadOneMessageOk(int len, int groupId) { 45 | logger.debug("NetworkBP tcpReadOneMessageOk, group : {}.", groupId); 46 | } 47 | 48 | public void tcpOnReadMessageLenError(int groupId) { 49 | logger.debug("NetworkBP tcpOnReadMessageLenError, group : {}.", groupId); 50 | } 51 | 52 | public void tcpReconnect(int groupId) { 53 | logger.debug("NetworkBP tcpReconnect, group : {}.", groupId); 54 | } 55 | 56 | public void tcpOutQueue(int delayMs, int groupId) { 57 | logger.debug("NetworkBP tcpOutQueue delayMs= {}.", delayMs); 58 | } 59 | 60 | public void sendRejectByTooLargeSize(int groupId) { 61 | logger.debug("NetworkBP sendRejectByTooLargeSize, group {}.", groupId); 62 | } 63 | 64 | public void send(byte[] message, int groupId) { 65 | logger.debug("NetworkBP send message length = {}.", (message == null ? 0 : message.length)); 66 | } 67 | 68 | public void sendTcp(byte[] message, int groupId) { 69 | logger.debug("NetworkBP sendTcp message length = {}.", (message == null ? 0 : message.length)); 70 | } 71 | 72 | public void sendUdp(byte[] message, int groupId) { 73 | logger.debug("NetworkBP sendUdp message length = {}.", (message == null ? 0 : message.length)); 74 | } 75 | 76 | public void sendMessageNodeIDNotFound(int groupId) { 77 | logger.debug("NetworkBP sendMessageNodeIDNotFound, group : {}.", groupId); 78 | } 79 | 80 | public void udpReceive(int recvLen, int groupId) { 81 | logger.debug("NetworkBP udpReceive recvLen= {}.", recvLen); 82 | } 83 | 84 | public void udpRealSend(String message, int groupId) { 85 | logger.debug("NetworkBP udpRealSend message lenth = {}.", (message == null ? 0 : message.length())); 86 | } 87 | 88 | public void udpQueueFull(int groupId) { 89 | logger.debug("NetworkBP udpQueueFull, group : {}.", groupId); 90 | } 91 | 92 | } 93 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/CheckpointMsgType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * checkpoint同步消息类型 20 | */ 21 | public enum CheckpointMsgType { 22 | sendFile(1), 23 | sendFileAck(2); 24 | 25 | private int value; 26 | 27 | private CheckpointMsgType(int value) { 28 | this.value = value; 29 | } 30 | 31 | public int getValue() { 32 | return value; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/CheckpointSendFileAckFlag.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * checkpoint同步ack状态标志 20 | */ 21 | public enum CheckpointSendFileAckFlag { 22 | OK(1), 23 | FAIL(2); 24 | 25 | private int value; 26 | 27 | private CheckpointSendFileAckFlag(int value) { 28 | this.value = value; 29 | } 30 | 31 | public int getValue() { 32 | return value; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/CheckpointSendFileFlag.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * checkpoint同步标志 20 | */ 21 | public enum CheckpointSendFileFlag { 22 | BEGIN(1), 23 | ING(2), 24 | END(3); 25 | 26 | private int value; 27 | 28 | private CheckpointSendFileFlag(int value) { 29 | this.value = value; 30 | } 31 | 32 | public int getValue() { 33 | return value; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/IndexType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * 索引存储类型 20 | */ 21 | public enum IndexType { 22 | LEVEL_DB(0), PHYSIC_FILE(1); 23 | private int type; 24 | 25 | IndexType(int type) { 26 | this.type = type; 27 | } 28 | 29 | public int getType() { 30 | return type; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/MessageSendType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * 消息发送方式 20 | */ 21 | public enum MessageSendType { 22 | UDP(0), 23 | TCP(1); 24 | 25 | private int value; 26 | 27 | private MessageSendType(int value) { 28 | this.value = value; 29 | } 30 | 31 | public int getValue() { 32 | return value; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/MsgCmd.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * 消息类型 20 | */ 21 | public enum MsgCmd { 22 | paxosMsg(1), 23 | checkpointMsg(2); 24 | 25 | private int value; 26 | 27 | private MsgCmd(int value) { 28 | this.value = value; 29 | } 30 | 31 | public int getValue() { 32 | return value; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/PaxosLogCleanType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * 历史数据清理方式 20 | */ 21 | public enum PaxosLogCleanType { 22 | cleanByHoldCount(1), 23 | cleanByTime(2); 24 | 25 | private int type; 26 | 27 | private PaxosLogCleanType(int type) { 28 | this.type = type; 29 | } 30 | 31 | public int getType() { 32 | return type; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/PaxosMsgFlagType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | public enum PaxosMsgFlagType { 19 | sendLearnValueNeedAck(1); 20 | 21 | private int value; 22 | 23 | private PaxosMsgFlagType(int value) { 24 | this.value = value; 25 | } 26 | 27 | public int getValue() { 28 | return value; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/PaxosMsgType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * paxosMsg协议类型 20 | */ 21 | public enum PaxosMsgType { 22 | paxosPrepare(1), 23 | paxosPrepareReply(2), 24 | paxosAccept(3), 25 | paxosAcceptReply(4), 26 | paxosLearnerAskforLearn(5), 27 | paxosLearnerSendLearnValue(6), 28 | paxosLearnerProposerSendSuccess(7), 29 | paxosProposalSendNewValue(8), 30 | paxosLearnerSendNowInstanceID(9), 31 | paxosLearnerComfirmAskforLearn(10), 32 | paxosLearnerSendLearnValueAck(11), 33 | paxosLearnerAskforCheckpoint(12), 34 | paxosLearnerOnAskforCheckpoint(13); 35 | 36 | private int value; 37 | 38 | private PaxosMsgType(int value) { 39 | this.value = value; 40 | } 41 | 42 | public int getValue() { 43 | return value; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/comm/enums/TimerType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.comm.enums; 17 | 18 | /** 19 | * 定时任务类型 20 | */ 21 | public enum TimerType { 22 | proposerPrepareTimeout(1), 23 | proposerAcceptTimeout(2), 24 | learnerAskforlearnNoop(3), 25 | instanceCommitTimeout(4); 26 | 27 | private int value; 28 | 29 | private TimerType(int value) { 30 | this.value = value; 31 | } 32 | 33 | public int getValue() { 34 | return value; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/NetWork.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.comm.NodeInfo; 22 | import com.wuba.wpaxos.node.Node; 23 | 24 | import java.util.Set; 25 | 26 | /** 27 | * 网络通信接口 28 | */ 29 | public abstract class NetWork { 30 | private final static Logger logger = LogManager.getLogger(NetWork.class); 31 | 32 | private Node node; 33 | 34 | // init and run network 35 | public abstract void runNetWork() throws Exception; 36 | 37 | // stop receive any message. 38 | public abstract void stopNetWork(); 39 | 40 | // send message by tcp 41 | public abstract int sendMessageTCP(int groupIdx, String ip, int port, byte[] message); 42 | 43 | // send message by udp 44 | public abstract int sendMessageUDP(int groupIdx, String ip, int port, byte[] message); 45 | 46 | // check conn keepalive nodelist 47 | public abstract void setCheckNode(int group, Set nodeInfo); 48 | 49 | //When receive a message, call this funtion. 50 | //This funtion is async, just enqueue an return. 51 | public int onReceiveMessage(ReceiveMessage receiveMsg) { 52 | if (this.node != null) { 53 | this.node.onReceiveMessage(receiveMsg); 54 | } else { 55 | logger.info("receive msglen {}, node null.", receiveMsg.getReceiveLen()); 56 | } 57 | 58 | return 0; 59 | } 60 | 61 | public Node getNode() { 62 | return node; 63 | } 64 | 65 | public void setNode(Node node) { 66 | this.node = node; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/ReceiveMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate; 17 | 18 | import org.jboss.netty.channel.Channel; 19 | 20 | /** 21 | * 接收数据封装 22 | */ 23 | public class ReceiveMessage { 24 | private byte[] receiveBuf; 25 | private int receiveLen; 26 | private boolean notifyMsg = false; 27 | private Channel channel; 28 | private long timeStamp; 29 | 30 | public ReceiveMessage(byte[] receiveBuf, int receiveLen) { 31 | super(); 32 | this.receiveBuf = receiveBuf; 33 | this.receiveLen = receiveLen; 34 | } 35 | 36 | public ReceiveMessage(byte[] receiveBuf, int receiveLen, boolean notifyMsg) { 37 | super(); 38 | this.receiveBuf = receiveBuf; 39 | this.receiveLen = receiveLen; 40 | this.notifyMsg = notifyMsg; 41 | } 42 | 43 | public static ReceiveMessage getNotifyNullMsg() { 44 | ReceiveMessage receiveMsg = new ReceiveMessage(null, 0, true); 45 | return receiveMsg; 46 | } 47 | 48 | public boolean isNotifyMsg() { 49 | return notifyMsg; 50 | } 51 | 52 | public void setNotifyMsg(boolean notifyMsg) { 53 | this.notifyMsg = notifyMsg; 54 | } 55 | 56 | public byte[] getReceiveBuf() { 57 | return receiveBuf; 58 | } 59 | 60 | public void setReceiveBuf(byte[] receiveBuf) { 61 | this.receiveBuf = receiveBuf; 62 | } 63 | 64 | public int getReceiveLen() { 65 | return receiveLen; 66 | } 67 | 68 | public void setReceiveLen(int receiveLen) { 69 | this.receiveLen = receiveLen; 70 | } 71 | 72 | public Channel getChannel() { 73 | return channel; 74 | } 75 | 76 | public void setChannel(Channel channel) { 77 | this.channel = channel; 78 | } 79 | 80 | public long getTimeStamp() { 81 | return timeStamp; 82 | } 83 | 84 | public void setTimeStamp(long timeStamp) { 85 | this.timeStamp = timeStamp; 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/client/IClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.client; 17 | 18 | import org.jboss.netty.channel.Channel; 19 | 20 | /** 21 | * client for send msg 22 | */ 23 | public interface IClient { 24 | 25 | /** 26 | * 初始化bootstrap 27 | */ 28 | public void init(); 29 | 30 | /** 31 | * 创建指定端口的连接 32 | * @param server 33 | * @param port 34 | * @return 35 | */ 36 | public Channel getChannel(String server, int port) throws Exception; 37 | } 38 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/client/tcp/TcpClientHander.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.client.tcp; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | import org.jboss.netty.channel.ChannelEvent; 21 | import org.jboss.netty.channel.ChannelHandlerContext; 22 | import org.jboss.netty.channel.ChannelStateEvent; 23 | import org.jboss.netty.channel.ExceptionEvent; 24 | import org.jboss.netty.channel.MessageEvent; 25 | import org.jboss.netty.channel.SimpleChannelUpstreamHandler; 26 | import org.jboss.netty.channel.ChannelHandler.Sharable; 27 | 28 | /** 29 | * TcpClient netty Hander 30 | */ 31 | @Sharable 32 | public class TcpClientHander extends SimpleChannelUpstreamHandler { 33 | private static final Logger logger = LogManager.getLogger(TcpClientHander.class); 34 | 35 | @Override 36 | public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { 37 | } 38 | 39 | @Override 40 | public void handleUpstream(ChannelHandlerContext ctx, ChannelEvent e) throws Exception { 41 | if (e instanceof ChannelStateEvent) { 42 | logger.debug(e.toString()); 43 | } 44 | super.handleUpstream(ctx, e); 45 | } 46 | 47 | @Override 48 | public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { 49 | logger.info("new channel open: {}.", e.getChannel().getId()); 50 | } 51 | 52 | @Override 53 | public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { 54 | logger.error("unexpected exception from downstream remoteAddress (" + e.getChannel().getRemoteAddress() + ")", e.getCause()); 55 | } 56 | 57 | @Override 58 | public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) { 59 | logger.info("channel is closed: {}.", e.getChannel().getRemoteAddress()); 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/client/udp/UDPClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.client.udp; 17 | 18 | import java.io.IOException; 19 | import java.net.DatagramPacket; 20 | import java.net.DatagramSocket; 21 | import java.net.InetSocketAddress; 22 | import java.net.SocketException; 23 | import java.nio.ByteBuffer; 24 | 25 | /** 26 | * udp client for send msg 27 | */ 28 | public class UDPClient { 29 | 30 | private String encode; 31 | 32 | private DatagramSocket sock = null; 33 | 34 | private InetSocketAddress addr = null; 35 | 36 | public static UDPClient getInstrance(String ip, int port, String encode) throws SocketException { 37 | UDPClient client = new UDPClient(); 38 | client.encode = encode; 39 | client.sock = new DatagramSocket(); 40 | client.addr = new InetSocketAddress(ip, port); 41 | 42 | return client; 43 | } 44 | 45 | private UDPClient() { 46 | } 47 | 48 | public void close() { 49 | sock.close(); 50 | } 51 | 52 | /** 53 | * send udp msg 54 | * @param msg 55 | */ 56 | public void send(String msg, String encoding) throws Exception { 57 | byte[] buf = msg.getBytes(encoding); 58 | send(buf); 59 | } 60 | 61 | public void send(String msg, byte[] delimiter) throws IOException { 62 | ByteBuffer buffer = ByteBuffer.allocate(msg.getBytes(encode).length + delimiter.length); 63 | buffer.put(msg.getBytes(encode)); 64 | buffer.put(delimiter); 65 | buffer.flip(); 66 | send(buffer.array()); 67 | buffer.clear(); 68 | } 69 | 70 | public void send(byte[] msg, byte[] delimiter) throws IOException { 71 | ByteBuffer buffer = ByteBuffer.allocate(msg.length + delimiter.length); 72 | buffer.put(msg); 73 | buffer.put(delimiter); 74 | buffer.flip(); 75 | send(buffer.array()); 76 | buffer.clear(); 77 | } 78 | 79 | public void send(String msg) throws IOException { 80 | byte[] buf = msg.getBytes(encode); 81 | send(buf); 82 | } 83 | 84 | public void send(byte[] buf) throws IOException { 85 | DatagramPacket dp = new DatagramPacket(buf, buf.length, addr); 86 | sock.send(dp); 87 | } 88 | } -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/server/IServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.server; 17 | 18 | public interface IServer { 19 | 20 | public void start() throws Exception; 21 | 22 | public void stop() throws Exception; 23 | } 24 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/server/tcp/TcpHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.server.tcp; 17 | 18 | import java.nio.ByteBuffer; 19 | 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | import org.jboss.netty.buffer.ChannelBuffer; 23 | import org.jboss.netty.channel.ChannelEvent; 24 | import org.jboss.netty.channel.ChannelHandlerContext; 25 | import org.jboss.netty.channel.ChannelStateEvent; 26 | import org.jboss.netty.channel.ExceptionEvent; 27 | import org.jboss.netty.channel.MessageEvent; 28 | import org.jboss.netty.channel.SimpleChannelUpstreamHandler; 29 | 30 | import com.wuba.wpaxos.communicate.ReceiveMessage; 31 | import com.wuba.wpaxos.communicate.config.ServerConfig; 32 | 33 | /** 34 | * Tcp server handler 35 | */ 36 | public class TcpHandler extends SimpleChannelUpstreamHandler { 37 | private static final Logger logger = LogManager.getLogger(TcpHandler.class); 38 | 39 | @Override 40 | public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { 41 | ByteBuffer buffer = ((ChannelBuffer) e.getMessage()).toByteBuffer(); 42 | byte[] reciveByte = buffer.array(); 43 | ReceiveMessage receiveMsg = new ReceiveMessage(reciveByte, reciveByte.length); 44 | receiveMsg.setChannel(e.getChannel()); 45 | receiveMsg.setTimeStamp(System.currentTimeMillis()); 46 | ServerConfig.getInstance().getNetwork().onReceiveMessage(receiveMsg); 47 | } 48 | 49 | @Override 50 | public void handleUpstream(ChannelHandlerContext ctx, ChannelEvent e) throws Exception { 51 | if (e instanceof ChannelStateEvent) { 52 | logger.debug(e.toString()); 53 | } 54 | super.handleUpstream(ctx, e); 55 | } 56 | 57 | @Override 58 | public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { 59 | TcpServer.allChannels.add(e.getChannel()); 60 | if (e.getChannel() != null && e.getChannel().getRemoteAddress() != null) { 61 | logger.info("new channel open: {}.", e.getChannel().getId()); 62 | } 63 | } 64 | 65 | @Override 66 | public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { 67 | logger.error("unexpected exception from downstream remoteAddress(" + e.getChannel().getRemoteAddress().toString() + ")", e.getCause()); 68 | } 69 | 70 | @Override 71 | public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) { 72 | logger.info("channel is closed: {}.", e.getChannel().getRemoteAddress().toString()); 73 | TcpServer.allChannels.remove(e.getChannel()); 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/server/tcp/TcpPipelineFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.server.tcp; 17 | 18 | import static org.jboss.netty.buffer.ChannelBuffers.directBuffer; 19 | import static org.jboss.netty.channel.Channels.pipeline; 20 | 21 | import org.jboss.netty.buffer.ChannelBuffer; 22 | import org.jboss.netty.channel.ChannelHandler; 23 | import org.jboss.netty.channel.ChannelPipeline; 24 | import org.jboss.netty.channel.ChannelPipelineFactory; 25 | import org.jboss.netty.handler.codec.frame.DelimiterBasedFrameDecoder; 26 | 27 | import com.wuba.wpaxos.proto.ProtocolConst; 28 | 29 | /** 30 | * TcpPipelineFactory 31 | */ 32 | public class TcpPipelineFactory implements ChannelPipelineFactory { 33 | private final ChannelHandler handler; 34 | private int frameMaxLength; 35 | private static ChannelBuffer buf = directBuffer(ProtocolConst.DELIMITER.length); 36 | 37 | static { 38 | buf.writeBytes(ProtocolConst.DELIMITER); 39 | } 40 | 41 | public TcpPipelineFactory(ChannelHandler handler, int frameMaxLength) { 42 | this.handler = handler; 43 | this.frameMaxLength = frameMaxLength; 44 | } 45 | 46 | @Override 47 | public ChannelPipeline getPipeline() throws Exception { 48 | ChannelPipeline pipeline = pipeline(); 49 | pipeline.addLast("framer", new DelimiterBasedFrameDecoder(this.frameMaxLength, true, buf)); 50 | pipeline.addLast("handler", handler); 51 | return pipeline; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/server/udp/UdpHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.server.udp; 17 | 18 | import java.nio.ByteBuffer; 19 | 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | import org.jboss.netty.buffer.ChannelBuffer; 23 | import org.jboss.netty.channel.ChannelEvent; 24 | import org.jboss.netty.channel.ChannelHandlerContext; 25 | import org.jboss.netty.channel.ChannelStateEvent; 26 | import org.jboss.netty.channel.ExceptionEvent; 27 | import org.jboss.netty.channel.MessageEvent; 28 | import org.jboss.netty.channel.SimpleChannelUpstreamHandler; 29 | import org.jboss.netty.channel.ChannelHandler.Sharable; 30 | 31 | import com.wuba.wpaxos.communicate.ReceiveMessage; 32 | import com.wuba.wpaxos.communicate.config.ServerConfig; 33 | 34 | /** 35 | * udp server hander for receive msg 36 | */ 37 | @Sharable 38 | public class UdpHandler extends SimpleChannelUpstreamHandler { 39 | private static final Logger logger = LogManager.getLogger(UdpHandler.class); 40 | 41 | @Override 42 | public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { 43 | try { 44 | ByteBuffer buffer = ((ChannelBuffer)e.getMessage()).toByteBuffer(); 45 | byte[] reciveByte = buffer.array(); 46 | ReceiveMessage receiveMsg = new ReceiveMessage(reciveByte, reciveByte.length); 47 | receiveMsg.setTimeStamp(System.currentTimeMillis()); 48 | ServerConfig.getInstance().getNetwork().onReceiveMessage(receiveMsg); 49 | } catch(Exception ex) { 50 | logger.error("UdpHandler messageReceived error.", ex); 51 | } 52 | } 53 | 54 | @Override 55 | public void handleUpstream(ChannelHandlerContext ctx, ChannelEvent e) throws Exception { 56 | super.handleUpstream(ctx, e); 57 | } 58 | 59 | @Override 60 | public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e) { 61 | UdpServer.allChannels.add(e.getChannel()); 62 | logger.info("udp channel open : {}.", ctx.getChannel().getRemoteAddress()); 63 | } 64 | 65 | @Override 66 | public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { 67 | } 68 | 69 | @Override 70 | public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { 71 | e.getChannel().close(); 72 | logger.error("unexpected exception from downstream remoteAddress(" + e.getChannel().getRemoteAddress().toString() + ")", e.getCause()); 73 | } 74 | 75 | @Override 76 | public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e){ 77 | e.getChannel().close(); 78 | UdpServer.allChannels.remove(e.getChannel()); 79 | } 80 | } -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/communicate/server/udp/UdpPipelineFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.communicate.server.udp; 17 | 18 | import static org.jboss.netty.buffer.ChannelBuffers.directBuffer; 19 | import static org.jboss.netty.channel.Channels.pipeline; 20 | 21 | import org.jboss.netty.buffer.ChannelBuffer; 22 | import org.jboss.netty.channel.ChannelHandler; 23 | import org.jboss.netty.channel.ChannelPipeline; 24 | import org.jboss.netty.channel.ChannelPipelineFactory; 25 | import org.jboss.netty.handler.codec.frame.DelimiterBasedFrameDecoder; 26 | 27 | import com.wuba.wpaxos.proto.ProtocolConst; 28 | 29 | /** 30 | * UdpPipelineFactory 31 | */ 32 | public class UdpPipelineFactory implements ChannelPipelineFactory { 33 | private static ChannelBuffer delimiterBuf = directBuffer(ProtocolConst.DELIMITER.length); 34 | 35 | static { 36 | delimiterBuf.writeBytes(ProtocolConst.DELIMITER); 37 | } 38 | 39 | private final ChannelHandler handler; 40 | private int frameMaxLength; 41 | 42 | public UdpPipelineFactory(ChannelHandler handler, int frameMaxLength) { 43 | this.handler = handler; 44 | this.frameMaxLength = frameMaxLength; 45 | } 46 | 47 | @Override 48 | public ChannelPipeline getPipeline() throws Exception { 49 | ChannelPipeline pipeline = pipeline(); 50 | pipeline.addLast("framer", new DelimiterBasedFrameDecoder(this.frameMaxLength, true, delimiterBuf)); 51 | pipeline.addLast("handler", handler); 52 | return pipeline; 53 | } 54 | } -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/config/Def.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.config; 17 | 18 | /** 19 | * 全局静态配置 20 | */ 21 | public class Def { 22 | public static final int SYSTEM_V_SMID = 100000000; 23 | public static final int MASTER_V_SMID = 100000001; 24 | public static final int BATCH_PROPOSE_SMID = 100000002; 25 | 26 | public static final int MAX_QUEUE_MEM = 209715200; 27 | public static final int CRC32SKIP = 4; // TODO check 28 | } 29 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/config/PaxosNodeFunctionRet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.config; 17 | 18 | /** 19 | * Paxos运行过程异常状态 20 | */ 21 | public enum PaxosNodeFunctionRet { 22 | Paxos_SystemError(-1), 23 | Paxos_GroupIdxWrong(-5), 24 | Paxos_MembershipOp_GidNotSame(-501), 25 | Paxos_MembershipOp_VersionConflit(-502), 26 | Paxos_MembershipOp_NoGid(1001), 27 | Paxos_MembershipOp_Add_NodeExist(1002), 28 | Paxos_MembershipOp_Remove_NodeNotExist(1003), 29 | Paxos_MembershipOp_Change_NoChange(1004), 30 | Paxos_GetInstanceValue_Value_NotExist(1005), 31 | Paxos_GetInstanceValue_Value_Not_Chosen_Yet(1006); 32 | 33 | private int ret; 34 | 35 | private PaxosNodeFunctionRet(int ret) { 36 | this.ret = ret; 37 | } 38 | 39 | public int getRet() { 40 | return ret; 41 | } 42 | 43 | public void setRet(int ret) { 44 | this.ret = ret; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/config/PaxosTryCommitRet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.config; 17 | 18 | /** 19 | * paxos commit结果状态 20 | */ 21 | public enum PaxosTryCommitRet { 22 | PaxosTryCommitRet_OK(0), 23 | PaxosTryCommitRet_Reject(-2), 24 | PaxosTryCommitRet_Conflict(14), 25 | PaxosTryCommitRet_ExecuteFail(15), 26 | PaxosTryCommitRet_Follower_Cannot_Commit(16), 27 | PaxosTryCommitRet_Im_Not_In_Membership(17), 28 | PaxosTryCommitRet_Value_Size_TooLarge(18), 29 | PaxosTryCommitRet_Timeout(404), 30 | PaxosTryCommitRet_TooManyThreadWaiting_Reject(405); 31 | 32 | private int ret; 33 | 34 | private PaxosTryCommitRet(int ret) { 35 | this.ret = ret; 36 | } 37 | 38 | public int getRet() { 39 | return ret; 40 | } 41 | 42 | public void setRet(int ret) { 43 | this.ret = ret; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/config/WriteOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.config; 17 | 18 | /** 19 | * 同步/异步刷盘配置选项 20 | */ 21 | public class WriteOptions { 22 | private boolean sync = true; 23 | 24 | public WriteOptions () {} 25 | 26 | public WriteOptions(boolean sync) { 27 | super(); 28 | this.sync = sync; 29 | } 30 | 31 | public boolean isSync() { 32 | return sync; 33 | } 34 | 35 | public void setSync(boolean sync) { 36 | this.sync = sync; 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/config/WriteState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.config; 17 | 18 | /** 19 | * 写入是否携带instance有效数据 20 | */ 21 | public class WriteState { 22 | private boolean hasPayLoad = false; 23 | private long instanceID; 24 | 25 | public WriteState() {} 26 | 27 | public WriteState(boolean hasPayLoad) { 28 | this.hasPayLoad = hasPayLoad; 29 | } 30 | 31 | public WriteState(boolean hasPayLoad, long instanceID) { 32 | super(); 33 | this.hasPayLoad = hasPayLoad; 34 | this.instanceID = instanceID; 35 | } 36 | 37 | public boolean isHasPayLoad() { 38 | return hasPayLoad; 39 | } 40 | 41 | public void setHasPayLoad(boolean hasPayLoad) { 42 | this.hasPayLoad = hasPayLoad; 43 | } 44 | 45 | public long getInstanceID() { 46 | return instanceID; 47 | } 48 | 49 | public void setInstanceID(long instanceID) { 50 | this.instanceID = instanceID; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/exception/SerializeException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.exception; 17 | 18 | /** 19 | * serialize/deserialize Exception 20 | */ 21 | public class SerializeException extends Exception { 22 | 23 | private static final long serialVersionUID = 1L; 24 | 25 | public SerializeException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public SerializeException(String message) { 30 | super(message); 31 | } 32 | 33 | public SerializeException(Throwable cause) { 34 | super("serialize exception", cause); 35 | } 36 | 37 | public SerializeException() { 38 | super("serialize exception"); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/helper/LearnerState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.helper; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.base.BallotNumber; 22 | import com.wuba.wpaxos.config.Config; 23 | import com.wuba.wpaxos.config.Def; 24 | import com.wuba.wpaxos.config.WriteOptions; 25 | import com.wuba.wpaxos.proto.AcceptorStateData; 26 | import com.wuba.wpaxos.store.LogStorage; 27 | import com.wuba.wpaxos.store.PaxosLog; 28 | import com.wuba.wpaxos.utils.Crc32; 29 | 30 | /** 31 | * leaner状态封装 32 | */ 33 | public class LearnerState { 34 | private static final Logger logger = LogManager.getLogger(LearnerState.class); 35 | private byte[] learnedValue; 36 | private boolean isLearned; 37 | private int newChecksum; 38 | private Config config; 39 | private PaxosLog paxosLog; 40 | 41 | public LearnerState(Config config, LogStorage logStorage) { 42 | this.paxosLog = new PaxosLog(logStorage); 43 | this.config = config; 44 | init(); 45 | } 46 | 47 | public void init() { 48 | this.learnedValue = new byte[]{}; 49 | this.isLearned = false; 50 | this.newChecksum = 0; 51 | } 52 | 53 | public int learnValue(long instanceID, BallotNumber learnedBallot, byte[] value, int lastChecksum) { 54 | if(instanceID > 0 && lastChecksum == 0) { 55 | this.newChecksum = 0; 56 | } else if(value.length > 0) { 57 | this.newChecksum = Crc32.crc32(lastChecksum, value, value.length, Def.CRC32SKIP); 58 | } 59 | 60 | AcceptorStateData state = new AcceptorStateData(); 61 | state.setInstanceID(instanceID); 62 | state.setAcceptedValue(value); 63 | state.setPromiseID(learnedBallot.getProposalID()); 64 | state.setPromiseNodeID(learnedBallot.getNodeId()); 65 | state.setAcceptedID(learnedBallot.getProposalID()); 66 | state.setAcceptedNodeID(learnedBallot.getNodeId()); 67 | state.setCheckSum(this.newChecksum); 68 | 69 | WriteOptions writeOptions = new WriteOptions(); 70 | writeOptions.setSync(false); 71 | 72 | int ret = this.paxosLog.writeState(writeOptions, this.config.getMyGroupIdx(), instanceID, state); 73 | 74 | if(ret != 0) { 75 | logger.error("LogStorage.WriteLog fail, InstanceID {}, ValueLen {}, ret {}.", instanceID, value.length, ret); 76 | return ret; 77 | } 78 | 79 | learnValueWithoutWrite(instanceID, value, this.newChecksum); 80 | return 0; 81 | } 82 | 83 | public void learnValueWithoutWrite(long instanceID, byte[] value, int newCheckSum) { 84 | this.learnedValue = value; 85 | this.isLearned = true; 86 | this.newChecksum = newCheckSum; 87 | } 88 | 89 | public byte[] getLearnValue() { 90 | return this.learnedValue; 91 | } 92 | 93 | public boolean getIsLearned() { 94 | return this.isLearned; 95 | } 96 | 97 | public int getNewChecksum() { 98 | return this.newChecksum; 99 | } 100 | } 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/helper/MsgCounter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.helper; 17 | 18 | import java.util.HashSet; 19 | import java.util.Set; 20 | 21 | import org.apache.logging.log4j.LogManager; 22 | import org.apache.logging.log4j.Logger; 23 | 24 | import com.wuba.wpaxos.config.Config; 25 | 26 | /** 27 | * propose提交过程,集群节点ack状态统计 28 | */ 29 | public class MsgCounter { 30 | private static final Logger logger = LogManager.getLogger(MsgCounter.class); 31 | public Config config; 32 | public Set receiveMsgNodeID = new HashSet(); 33 | public Set rejectMsgNodeID = new HashSet(); 34 | public Set promiseOrAcceptMsgNodeID = new HashSet(); 35 | 36 | public MsgCounter(Config config) { 37 | this.config = config; 38 | this.startNewRound(); 39 | } 40 | 41 | public void addReceive(Long nodeID) { 42 | if(!this.receiveMsgNodeID.contains(nodeID)) { 43 | this.receiveMsgNodeID.add(nodeID); 44 | } 45 | } 46 | 47 | public void addReject(long nodeID) { 48 | if(!this.rejectMsgNodeID.contains(nodeID)) { 49 | this.rejectMsgNodeID.add(nodeID); 50 | } 51 | } 52 | 53 | public void addPromiseOrAccept(long nodeID) { 54 | if(!this.promiseOrAcceptMsgNodeID.contains(nodeID)) { 55 | this.promiseOrAcceptMsgNodeID.add(nodeID); 56 | } 57 | } 58 | 59 | public boolean isPassedOnThisRound() { 60 | logger.debug("passedOn size {}.", this.promiseOrAcceptMsgNodeID.size()); 61 | return this.promiseOrAcceptMsgNodeID.size() >= this.config.getMajorityCount(); 62 | } 63 | 64 | public boolean isRejectedOnThisRound() { 65 | return this.rejectMsgNodeID.size() >= this.config.getMajorityCount(); 66 | } 67 | 68 | public boolean isAllReceiveOnThisRound() { 69 | return this.receiveMsgNodeID.size() == this.config.getNodeCount(); 70 | } 71 | 72 | public void startNewRound() { 73 | if(this.receiveMsgNodeID != null) { 74 | this.receiveMsgNodeID.clear(); 75 | } 76 | if(this.rejectMsgNodeID != null) { 77 | this.rejectMsgNodeID.clear(); 78 | } 79 | if(this.promiseOrAcceptMsgNodeID != null) { 80 | this.promiseOrAcceptMsgNodeID.clear(); 81 | } 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/helper/SerialLock.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.helper; 17 | 18 | import java.util.concurrent.TimeUnit; 19 | import java.util.concurrent.locks.Condition; 20 | import java.util.concurrent.locks.ReentrantLock; 21 | 22 | /** 23 | * 互斥锁封装 24 | */ 25 | public class SerialLock { 26 | private ReentrantLock lock = new ReentrantLock(); 27 | private Condition cond = lock.newCondition(); 28 | 29 | public void lock() { 30 | this.lock.lock(); 31 | } 32 | 33 | public void unLock() { 34 | if (this.lock.isLocked()) { 35 | this.lock.unlock(); 36 | } 37 | } 38 | 39 | public boolean isLocked() { 40 | return this.lock.isLocked(); 41 | } 42 | 43 | public void waitSomeMs() throws InterruptedException { 44 | this.cond.await(); 45 | } 46 | 47 | public void interupt() { 48 | this.cond.signal(); 49 | } 50 | 51 | public boolean waitTime(int timeMs) throws InterruptedException { 52 | return this.cond.await(timeMs, TimeUnit.MILLISECONDS); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/master/MasterInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.master; 17 | 18 | /** 19 | * master current info wrapper 20 | */ 21 | public class MasterInfo { 22 | private long masterNodeID = 0; 23 | private long masterVersion = 0; 24 | 25 | public MasterInfo() {} 26 | 27 | public MasterInfo(long masterNodeID, long masterVersion) { 28 | super(); 29 | this.masterNodeID = masterNodeID; 30 | this.masterVersion = masterVersion; 31 | } 32 | 33 | public long getMasterNodeID() { 34 | return masterNodeID; 35 | } 36 | 37 | public void setMasterNodeID(long masterNodeID) { 38 | this.masterNodeID = masterNodeID; 39 | } 40 | 41 | public long getMasterVersion() { 42 | return masterVersion; 43 | } 44 | 45 | public void setMasterVersion(long masterVersion) { 46 | this.masterVersion = masterVersion; 47 | } 48 | 49 | @Override 50 | public String toString() { 51 | return "MasterInfo [masterNodeID=" + masterNodeID + ", masterVersion=" + masterVersion + "]"; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/master/MasterVariablesStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.master; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.config.WriteOptions; 22 | import com.wuba.wpaxos.exception.SerializeException; 23 | import com.wuba.wpaxos.proto.MasterVariables; 24 | import com.wuba.wpaxos.store.LogStorage; 25 | 26 | /** 27 | * master info persist 28 | */ 29 | public class MasterVariablesStore { 30 | private final Logger logger = LogManager.getLogger(MasterVariablesStore.class); 31 | private LogStorage logStorage; 32 | 33 | public MasterVariablesStore(LogStorage logStorage) { 34 | super(); 35 | this.logStorage = logStorage; 36 | } 37 | 38 | public int write(WriteOptions writeOptions, int groupIdx, MasterVariables variables) { 39 | 40 | byte[] buffer = null; 41 | try { 42 | buffer = variables.serializeToBytes(); 43 | } catch (SerializeException e) { 44 | logger.error("MasterVariables serialize error.", e); 45 | } 46 | if (buffer == null) { 47 | logger.error("Variables serialize failed."); 48 | return -1; 49 | } 50 | int ret = logStorage.setMasterVariables(writeOptions, groupIdx, buffer); 51 | if (ret != 0) { 52 | logger.error("DB put failed, groupidx {}, bufferLen {} ret {}.", groupIdx, buffer.length, ret); 53 | return ret; 54 | } 55 | 56 | return 0; 57 | } 58 | 59 | public MasterVariables read(int groupIdx) { 60 | 61 | byte[] buffer = logStorage.getMasterVariables(groupIdx); 62 | if (buffer == null) { 63 | logger.error("DB get failed, groupIdx {}.", groupIdx); 64 | return null; 65 | } 66 | 67 | MasterVariables variables = new MasterVariables(); 68 | try { 69 | variables.parseFromBytes(buffer, buffer.length); 70 | } catch(Exception e) { 71 | logger.error("MasterVariables parse error.", e); 72 | return null; 73 | } 74 | 75 | return variables; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/BatchPaxosValue.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import java.util.ArrayList; 19 | import java.util.List; 20 | 21 | import com.google.protobuf.ByteString; 22 | import com.wuba.wpaxos.exception.SerializeException; 23 | import com.wuba.wpaxos.proto.pb.PaxosProto; 24 | 25 | /** 26 | * BatchPaxosValue proto 27 | */ 28 | public class BatchPaxosValue implements Proto { 29 | private List batchList = new ArrayList(); 30 | 31 | public int getBatchSize() { 32 | return batchList.size(); 33 | } 34 | 35 | public PaxosValue getByIndex(int idx) { 36 | if (idx >= batchList.size()) { 37 | return null; 38 | } 39 | return batchList.get(idx); 40 | } 41 | 42 | public void addPaxosValue(PaxosValue paxosValue) { 43 | this.batchList.add(paxosValue); 44 | } 45 | 46 | public List getBatchList() { 47 | return batchList; 48 | } 49 | 50 | public void setBatchList(List batchList) { 51 | this.batchList = batchList; 52 | } 53 | 54 | @Override 55 | public byte[] serializeToBytes() throws SerializeException { 56 | try { 57 | PaxosProto.BatchPaxosValues.Builder builder = PaxosProto.BatchPaxosValues.newBuilder(); 58 | for (PaxosValue paxosValue : batchList) { 59 | PaxosProto.PaxosValue.Builder bld = PaxosProto.PaxosValue.newBuilder(); 60 | bld.setSMID(paxosValue.getSmID()); 61 | bld.setValue(ByteString.copyFrom(paxosValue.getValue())); 62 | PaxosProto.PaxosValue pValue = bld.build(); 63 | builder.addValues(pValue); 64 | } 65 | return builder.build().toByteArray(); 66 | } catch(Exception e) { 67 | throw new SerializeException("BatchPaxosValue serializeToBytes failed.", e); 68 | } 69 | } 70 | 71 | @Override 72 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 73 | try { 74 | PaxosProto.BatchPaxosValues batchPaxosValue = PaxosProto.BatchPaxosValues.parseFrom(buf); 75 | List pValueList = batchPaxosValue.getValuesList(); 76 | for (PaxosProto.PaxosValue pValue : pValueList) { 77 | PaxosValue paxosValue = new PaxosValue(); 78 | paxosValue.setSmID(pValue.getSMID()); 79 | paxosValue.setValue(pValue.getValue().toByteArray()); 80 | this.batchList.add(paxosValue); 81 | } 82 | } catch(Exception e) { 83 | throw new SerializeException("Parse BatchPaxosValue failed.", e); 84 | } 85 | 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/Header.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.wuba.wpaxos.exception.SerializeException; 19 | import com.wuba.wpaxos.proto.pb.PaxosProto; 20 | 21 | /** 22 | * Proto header 23 | */ 24 | public class Header implements Proto { 25 | public static final int HEADERLEN = 24; 26 | 27 | private long gid; 28 | 29 | private long rid; 30 | 31 | private int cmdid; 32 | 33 | private int version; 34 | 35 | public long getGid() { 36 | return gid; 37 | } 38 | 39 | public void setGid(long gid) { 40 | this.gid = gid; 41 | } 42 | 43 | public long getRid() { 44 | return rid; 45 | } 46 | 47 | public void setRid(long rid) { 48 | this.rid = rid; 49 | } 50 | 51 | public int getCmdid() { 52 | return cmdid; 53 | } 54 | 55 | public void setCmdid(int cmdid) { 56 | this.cmdid = cmdid; 57 | } 58 | 59 | public int getVersion() { 60 | return version; 61 | } 62 | 63 | public void setVersion(int version) { 64 | this.version = version; 65 | } 66 | 67 | @Override 68 | public byte[] serializeToBytes() throws SerializeException { 69 | try { 70 | PaxosProto.Header.Builder builder = PaxosProto.Header.newBuilder(); 71 | builder.setGid(this.gid); 72 | builder.setRid(this.rid); 73 | builder.setCmdid(this.cmdid); 74 | builder.setVersion(this.version); 75 | PaxosProto.Header header = builder.build(); 76 | return header.toByteArray(); 77 | } catch (Exception e) { 78 | throw new SerializeException("Header serializeToBytes failed.", e); 79 | } 80 | } 81 | 82 | @Override 83 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 84 | try { 85 | PaxosProto.Header header = PaxosProto.Header.parseFrom(buf); 86 | this.setGid(header.getGid()); 87 | this.setRid(header.getRid()); 88 | this.setCmdid(header.getCmdid()); 89 | this.setVersion(header.getVersion()); 90 | } catch(Exception e) { 91 | throw new SerializeException("Parse Header failed.", e); 92 | } 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/MasterOperator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.wuba.wpaxos.exception.SerializeException; 19 | import com.wuba.wpaxos.proto.pb.MasterSmProto; 20 | 21 | /** 22 | * MasterOperator proto 23 | */ 24 | public class MasterOperator implements Proto { 25 | private long nodeID; 26 | private long version; 27 | private int timeout; 28 | private int operator; 29 | private int sid; 30 | private long lastversion; 31 | 32 | public long getNodeID() { 33 | return nodeID; 34 | } 35 | 36 | public void setNodeID(long nodeID) { 37 | this.nodeID = nodeID; 38 | } 39 | 40 | public long getVersion() { 41 | return version; 42 | } 43 | 44 | public void setVersion(long version) { 45 | this.version = version; 46 | } 47 | 48 | public int getTimeout() { 49 | return timeout; 50 | } 51 | 52 | public void setTimeout(int timeout) { 53 | this.timeout = timeout; 54 | } 55 | 56 | public int getOperator() { 57 | return operator; 58 | } 59 | 60 | public void setOperator(int operator) { 61 | this.operator = operator; 62 | } 63 | 64 | public int getSid() { 65 | return sid; 66 | } 67 | 68 | public void setSid(int sid) { 69 | this.sid = sid; 70 | } 71 | 72 | public long getLastversion() { 73 | return lastversion; 74 | } 75 | 76 | public void setLastversion(long lastversion) { 77 | this.lastversion = lastversion; 78 | } 79 | 80 | @Override 81 | public byte[] serializeToBytes() throws SerializeException { 82 | try { 83 | MasterSmProto.MasterOperator.Builder builder = MasterSmProto.MasterOperator.newBuilder(); 84 | builder.setNodeid(this.nodeID); 85 | builder.setVersion(this.version); 86 | builder.setTimeout(this.timeout); 87 | builder.setOperator(this.operator); 88 | builder.setSid(this.sid); 89 | builder.setLastversion(this.lastversion); 90 | MasterSmProto.MasterOperator masterOperator = builder.build(); 91 | 92 | return masterOperator.toByteArray(); 93 | } catch(Exception e) { 94 | throw new SerializeException("MasterOperator serializeToBytes failed.", e); 95 | } 96 | } 97 | 98 | @Override 99 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 100 | try { 101 | MasterSmProto.MasterOperator masterOperator = MasterSmProto.MasterOperator.parseFrom(buf); 102 | this.setNodeID(masterOperator.getNodeid()); 103 | this.setVersion(masterOperator.getVersion()); 104 | this.setTimeout(masterOperator.getTimeout()); 105 | this.setOperator(masterOperator.getOperator()); 106 | this.setSid(masterOperator.getSid()); 107 | this.setLastversion(masterOperator.getLastversion()); 108 | } catch(Exception e) { 109 | throw new SerializeException("Parse MasterOperator failed.", e); 110 | } 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/MasterVariables.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.wuba.wpaxos.exception.SerializeException; 19 | import com.wuba.wpaxos.proto.pb.PaxosProto; 20 | 21 | /** 22 | * MasterVariables proto 23 | */ 24 | public class MasterVariables implements Proto { 25 | private long masterNodeID; 26 | private long version; 27 | private int leaseTime; 28 | 29 | public long getMasterNodeID() { 30 | return masterNodeID; 31 | } 32 | 33 | public void setMasterNodeID(long masterNodeID) { 34 | this.masterNodeID = masterNodeID; 35 | } 36 | 37 | public long getVersion() { 38 | return version; 39 | } 40 | 41 | public void setVersion(long version) { 42 | this.version = version; 43 | } 44 | 45 | public int getLeaseTime() { 46 | return leaseTime; 47 | } 48 | 49 | public void setLeaseTime(int leaseTime) { 50 | this.leaseTime = leaseTime; 51 | } 52 | 53 | @Override 54 | public byte[] serializeToBytes() throws SerializeException { 55 | try { 56 | PaxosProto.MasterVariables.Builder builder = PaxosProto.MasterVariables.newBuilder(); 57 | builder.setMasterNodeid(this.masterNodeID); 58 | builder.setVersion(this.version); 59 | builder.setLeaseTime(this.leaseTime); 60 | PaxosProto.MasterVariables masterVariables = builder.build(); 61 | 62 | return masterVariables.toByteArray(); 63 | } catch(Exception e) { 64 | throw new SerializeException("MasterVariables serializeToBytes failed.", e); 65 | } 66 | } 67 | 68 | @Override 69 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 70 | try { 71 | PaxosProto.MasterVariables masterVariables = PaxosProto.MasterVariables.parseFrom(buf); 72 | this.setMasterNodeID(masterVariables.getMasterNodeid()); 73 | this.setLeaseTime(masterVariables.getLeaseTime()); 74 | this.setVersion(masterVariables.getVersion()); 75 | } catch(Exception e) { 76 | throw new SerializeException("Parse MasterVariables failed.", e); 77 | } 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/PaxosNodeInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.wuba.wpaxos.exception.SerializeException; 19 | import com.wuba.wpaxos.proto.pb.PaxosProto; 20 | 21 | /** 22 | * PaxosNodeInfo proto 23 | */ 24 | public class PaxosNodeInfo implements Proto { 25 | private long rid; 26 | private long nodeID; 27 | 28 | public long getRid() { 29 | return rid; 30 | } 31 | 32 | public void setRid(long rid) { 33 | this.rid = rid; 34 | } 35 | 36 | public long getNodeID() { 37 | return nodeID; 38 | } 39 | 40 | public void setNodeID(long nodeID) { 41 | this.nodeID = nodeID; 42 | } 43 | 44 | @Override 45 | public byte[] serializeToBytes() throws SerializeException { 46 | try { 47 | PaxosProto.PaxosNodeInfo.Builder builder = PaxosProto.PaxosNodeInfo.newBuilder(); 48 | builder.setRid(this.rid); 49 | builder.setNodeid(this.nodeID); 50 | PaxosProto.PaxosNodeInfo paxosNodeInfo = builder.build(); 51 | 52 | return paxosNodeInfo.toByteArray(); 53 | } catch(Exception e) { 54 | throw new SerializeException("PaxosNodeInfo serializeToBytes failed.", e); 55 | } 56 | } 57 | 58 | @Override 59 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 60 | try { 61 | PaxosProto.PaxosNodeInfo paxosNodeInfo = PaxosProto.PaxosNodeInfo.parseFrom(buf); 62 | this.setRid(paxosNodeInfo.getRid()); 63 | this.setNodeID(paxosNodeInfo.getNodeid()); 64 | } catch(Exception e) { 65 | throw new SerializeException("Parse PaxosNodeInfo failed.", e); 66 | } 67 | } 68 | 69 | @Override 70 | public String toString() { 71 | return "PaxosNodeInfo [rid=" + rid + ", nodeID=" + nodeID + "]"; 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/PaxosValue.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.google.protobuf.ByteString; 19 | import com.wuba.wpaxos.exception.SerializeException; 20 | import com.wuba.wpaxos.proto.pb.PaxosProto; 21 | 22 | /** 23 | * PaxosValue 24 | */ 25 | public class PaxosValue implements Proto { 26 | private int smID; 27 | private byte[] value; 28 | 29 | public int getSmID() { 30 | return smID; 31 | } 32 | 33 | public void setSmID(int smID) { 34 | this.smID = smID; 35 | } 36 | 37 | public byte[] getValue() { 38 | return value; 39 | } 40 | 41 | public void setValue(byte[] value) { 42 | this.value = value; 43 | } 44 | 45 | @Override 46 | public byte[] serializeToBytes() throws SerializeException { 47 | try { 48 | PaxosProto.PaxosValue.Builder builder = PaxosProto.PaxosValue.newBuilder(); 49 | builder.setSMID(this.smID); 50 | builder.setValue(ByteString.copyFrom(this.value)); 51 | PaxosProto.PaxosValue paxosValue = builder.build(); 52 | return paxosValue.toByteArray(); 53 | } catch (Exception e) { 54 | throw new SerializeException("PaxosValue serializeToBytes failed.", e); 55 | } 56 | } 57 | 58 | @Override 59 | public void parseFromBytes(byte[] buf, int len) throws SerializeException { 60 | try { 61 | PaxosProto.PaxosValue paxosValue = PaxosProto.PaxosValue.parseFrom(buf); 62 | this.setSmID(paxosValue.getSMID()); 63 | this.setValue(paxosValue.getValue().toByteArray()); 64 | 65 | } catch(Exception e) { 66 | throw new SerializeException("Parse AcceptorStatedata failed.", e); 67 | } 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/Proto.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | import com.wuba.wpaxos.exception.SerializeException; 19 | 20 | public interface Proto { 21 | 22 | public byte[] serializeToBytes() throws SerializeException ; 23 | 24 | public void parseFromBytes(byte[] buf, int len) throws SerializeException; 25 | 26 | } 27 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/ProtocolConst.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.proto; 17 | 18 | /** 19 | * ProtocolConst 20 | */ 21 | public class ProtocolConst { 22 | 23 | public static final byte[] P_START_TAG = new byte[]{18, 17, 13, 10, 9}; 24 | 25 | public static final byte[] P_END_TAG = new byte[]{9, 10, 13, 17, 18}; 26 | 27 | public static final byte[] DELIMITER = new byte[]{9, 10, 13, 17, 18}; 28 | } -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/pb/masterSmProto.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | package com.bj58.wpaxos.proto; 3 | 4 | message MasterOperator 5 | { 6 | required uint64 nodeid = 1; 7 | required uint64 version = 2; 8 | required int32 timeout = 3; 9 | required uint32 operator = 4; 10 | required uint32 sid = 5; 11 | optional uint64 lastversion = 6; 12 | }; 13 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/proto/pb/paxosProto.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto2"; 2 | package com.bj58.wpaxos.proto; 3 | 4 | message Header 5 | { 6 | required uint64 gid = 1; 7 | required uint64 rid = 2; 8 | required int32 cmdid = 3; 9 | optional int32 version = 4; 10 | }; 11 | 12 | message PaxosMsg 13 | { 14 | required int32 MsgType = 1; 15 | optional uint64 InstanceID = 2; 16 | optional uint64 NodeID = 3; 17 | optional uint64 ProposalID = 4; 18 | optional uint64 ProposalNodeID = 5; 19 | optional bytes Value = 6; 20 | optional uint64 PreAcceptID = 7; 21 | optional uint64 PreAcceptNodeID = 8; 22 | optional uint64 RejectByPromiseID = 9; 23 | optional uint64 NowInstanceID = 10; 24 | optional uint64 MinChosenInstanceID = 11; 25 | optional uint32 LastChecksum = 12; 26 | optional uint32 Flag = 13; 27 | optional bytes SystemVariables = 14; 28 | optional bytes MasterVariables = 15; 29 | optional uint64 TimeStamp = 16; 30 | }; 31 | 32 | message CheckpointMsg 33 | { 34 | required int32 MsgType = 1; 35 | required uint64 NodeID = 2; 36 | optional int32 Flag = 3; 37 | required uint64 UUID = 4; 38 | required uint64 Sequence = 5; 39 | optional uint64 CheckpointInstanceID = 6; 40 | optional uint32 Checksum = 7; 41 | optional string FilePath = 8; 42 | optional int32 SMID = 9; 43 | optional uint64 Offset = 10; 44 | optional bytes Buffer = 11; 45 | } 46 | 47 | message AcceptorStateData 48 | { 49 | required uint64 InstanceID = 1; 50 | required uint64 PromiseID = 2; 51 | required uint64 PromiseNodeID = 3; 52 | required uint64 AcceptedID = 4; 53 | required uint64 AcceptedNodeID = 5; 54 | required bytes AcceptedValue = 6; 55 | required uint32 Checksum = 7; 56 | }; 57 | 58 | message PaxosNodeInfo 59 | { 60 | required uint64 Rid = 1; 61 | required uint64 Nodeid = 2; 62 | }; 63 | 64 | message SystemVariables 65 | { 66 | required uint64 Gid = 1; 67 | repeated PaxosNodeInfo MemberShip = 2; 68 | required uint64 Version = 3; 69 | }; 70 | 71 | message MasterVariables 72 | { 73 | required uint64 MasterNodeid = 1; 74 | required uint64 Version = 2; 75 | required uint32 LeaseTime = 3; 76 | }; 77 | 78 | message PaxosValue 79 | { 80 | required int32 SMID = 1; 81 | required bytes Value = 2; 82 | }; 83 | 84 | message BatchPaxosValues 85 | { 86 | repeated PaxosValue Values = 1; 87 | }; 88 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/AppendDataCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | import java.nio.ByteBuffer; 19 | 20 | import com.wuba.wpaxos.config.WriteState; 21 | 22 | /** 23 | * append data to MapedByteBuffer 24 | */ 25 | public interface AppendDataCallback { 26 | 27 | /** 28 | * write MapedByteBuffer 29 | * 30 | * @param byteBuffer 31 | * @param maxBlank 32 | * @param data 33 | * @param writestate to decide msg type 34 | * @return How many bytes to write 35 | */ 36 | public AppendDataResult doAppend(final long fileFromOffset, final ByteBuffer byteBuffer, 37 | final int maxBlank, final byte[] data, WriteState writestate); 38 | } 39 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/AppendDataResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | /** 19 | * append data result 20 | */ 21 | public class AppendDataResult { 22 | // Return code 23 | private AppendDataStatus status; 24 | // Where to start writing 25 | private long wroteOffset; 26 | // Write Bytes 27 | private int wroteBytes; 28 | // Message storage timestamp 29 | private long storeTimestamp; 30 | private int crc32; 31 | 32 | public AppendDataResult(AppendDataStatus status) { 33 | this(status, 0, 0, -1L, 0); 34 | } 35 | 36 | public AppendDataResult(AppendDataStatus status, long wroteOffset, int wroteBytes, 37 | long storeTimestamp, int crc32) { 38 | this.status = status; 39 | this.wroteOffset = wroteOffset; 40 | this.wroteBytes = wroteBytes; 41 | this.storeTimestamp = storeTimestamp; 42 | this.crc32 = crc32; 43 | } 44 | 45 | public boolean isOk() { 46 | return this.status == AppendDataStatus.PUT_OK; 47 | } 48 | 49 | public AppendDataStatus getStatus() { 50 | return status; 51 | } 52 | 53 | public void setStatus(AppendDataStatus status) { 54 | this.status = status; 55 | } 56 | 57 | public long getWroteOffset() { 58 | return wroteOffset; 59 | } 60 | 61 | public void setWroteOffset(long wroteOffset) { 62 | this.wroteOffset = wroteOffset; 63 | } 64 | 65 | public int getWroteBytes() { 66 | return wroteBytes; 67 | } 68 | 69 | public void setWroteBytes(int wroteBytes) { 70 | this.wroteBytes = wroteBytes; 71 | } 72 | 73 | public long getStoreTimestamp() { 74 | return storeTimestamp; 75 | } 76 | 77 | public void setStoreTimestamp(long storeTimestamp) { 78 | this.storeTimestamp = storeTimestamp; 79 | } 80 | 81 | public int getCrc32() { 82 | return crc32; 83 | } 84 | 85 | public void setCrc32(int crc32) { 86 | this.crc32 = crc32; 87 | } 88 | 89 | @Override 90 | public String toString() { 91 | return "AppendDataResult [status=" + status + ", wroteOffset=" + wroteOffset + ", wroteBytes=" 92 | + wroteBytes + ", storeTimestamp=" + storeTimestamp + "]"; 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/AppendDataStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | /** 19 | * append data 返回状态码 20 | */ 21 | public enum AppendDataStatus { 22 | PUT_OK, 23 | END_OF_FILE, 24 | MESSAGE_SIZE_EXCEEDED, 25 | UNKNOWN_ERROR, 26 | } 27 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/GetResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | import java.nio.ByteBuffer; 19 | 20 | import com.wuba.wpaxos.store.pagecache.MapedFile; 21 | 22 | /** 23 | * get bytebuffer slice of mapedFile from startOffset 24 | */ 25 | public class GetResult { 26 | // 从队列中哪个绝对Offset开始 27 | private final long startOffset; 28 | // position从0开始 29 | private final ByteBuffer byteBuffer; 30 | // 有效数据大小 31 | private int size; 32 | // 用来释放内存 33 | private MapedFile mapedFile; 34 | 35 | public GetResult(long startOffset, ByteBuffer byteBuffer, int size, MapedFile mapedFile) { 36 | this.startOffset = startOffset; 37 | this.byteBuffer = byteBuffer; 38 | this.size = size; 39 | this.mapedFile = mapedFile; 40 | } 41 | 42 | public ByteBuffer getByteBuffer() { 43 | return byteBuffer; 44 | } 45 | 46 | public int getSize() { 47 | return size; 48 | } 49 | 50 | public void setSize(final int s) { 51 | this.size = s; 52 | this.byteBuffer.limit(this.size); 53 | } 54 | 55 | public MapedFile getMapedFile() { 56 | return mapedFile; 57 | } 58 | 59 | /** 60 | * 此方法只能被调用一次,重复调用无效 61 | */ 62 | public synchronized void release() { 63 | if (this.mapedFile != null) { 64 | this.mapedFile.release(); 65 | this.mapedFile = null; 66 | } 67 | } 68 | 69 | public long getStartOffset() { 70 | return startOffset; 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/PutDataResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | /** 19 | * put data result, instance file index and put status 20 | */ 21 | public class PutDataResult { 22 | private FileID fileID; 23 | private PutDataStatus putDataStatus; 24 | private AppendDataResult appendDataResult; 25 | 26 | public PutDataResult(PutDataStatus putDataStatus, AppendDataResult appendDataResult) { 27 | super(); 28 | this.putDataStatus = putDataStatus; 29 | this.appendDataResult = appendDataResult; 30 | } 31 | 32 | public FileID getFileID() { 33 | return fileID; 34 | } 35 | 36 | public void setFileID(FileID fileID) { 37 | this.fileID = fileID; 38 | } 39 | 40 | public PutDataStatus getPutDataStatus() { 41 | return putDataStatus; 42 | } 43 | 44 | public void setPutDataStatus(PutDataStatus putDataStatus) { 45 | this.putDataStatus = putDataStatus; 46 | } 47 | 48 | public AppendDataResult getAppendDataResult() { 49 | return appendDataResult; 50 | } 51 | 52 | public void setAppendDataResult(AppendDataResult appendDataResult) { 53 | this.appendDataResult = appendDataResult; 54 | } 55 | 56 | public boolean isOk() { 57 | return this.appendDataResult != null && this.appendDataResult.isOk(); 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "PutDataResult [putDataStatus=" + putDataStatus + ", appendDataResult=" + appendDataResult + "]"; 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/PutDataStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | public enum PutDataStatus { 19 | PUT_OK(0), 20 | FLUSH_DISK_TIMEOUT(1), 21 | CREATE_MAPEDFILE_FAILED(5), 22 | MESSAGE_ILLEGAL(6), 23 | UNKNOWN_ERROR(7); 24 | 25 | int value; 26 | 27 | PutDataStatus(int val) { 28 | this.value = val; 29 | } 30 | 31 | public int getValue() { 32 | return value; 33 | } 34 | 35 | public void setValue(int value) { 36 | this.value = value; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/SystemVariablesStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.config.WriteOptions; 22 | import com.wuba.wpaxos.exception.SerializeException; 23 | import com.wuba.wpaxos.proto.SystemVariables; 24 | 25 | /** 26 | * group membership storage 27 | */ 28 | public class SystemVariablesStore { 29 | private static final Logger logger = LogManager.getLogger(SystemVariablesStore.class); 30 | 31 | private LogStorage logStorage; 32 | 33 | public SystemVariablesStore(LogStorage logStorage) { 34 | super(); 35 | this.logStorage = logStorage; 36 | } 37 | 38 | public LogStorage getLogStorage() { 39 | return logStorage; 40 | } 41 | 42 | public void setLogStorage(LogStorage logStorage) { 43 | this.logStorage = logStorage; 44 | } 45 | 46 | public int write(WriteOptions writeOptions, int groupIdx, SystemVariables variables) { 47 | byte[] buf = null; 48 | try { 49 | buf = variables.serializeToBytes(); 50 | } catch (SerializeException e) { 51 | logger.error("System variables serialize failed.", e); 52 | return -1; 53 | } 54 | 55 | int ret = this.logStorage.setSystemVariables(writeOptions, groupIdx, buf); 56 | if (ret != 0) { 57 | logger.error("DB.Put failed, groupIdx {}, bufferlen {} ret {}.", groupIdx, buf.length, ret); 58 | return ret; 59 | } 60 | 61 | return 0; 62 | } 63 | 64 | public SystemVariables read(int groupIdx) { 65 | byte[] buf = this.logStorage.getSystemVariables(groupIdx); 66 | if(buf == null || buf.length == 0) return null; 67 | 68 | SystemVariables variables = new SystemVariables(); 69 | try { 70 | variables.parseFromBytes(buf, buf.length); 71 | } catch (SerializeException e) { 72 | logger.error("SystemVariables read error", e); 73 | return null; 74 | } 75 | return variables; 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/config/DynamicConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.config; 17 | 18 | /** 19 | * config load dynamic 20 | */ 21 | public interface DynamicConfig { 22 | 23 | /** 24 | * config will be load when the file modified 25 | * @throws Exception 26 | */ 27 | public void loadConfig() throws Exception; 28 | } 29 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/config/FlushDiskType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.wuba.wpaxos.store.config; 15 | 16 | /** 17 | * 刷盘方式 18 | */ 19 | public enum FlushDiskType { 20 | 21 | /** 22 | * 同步刷盘 23 | */ 24 | SYNC_FLUSH, 25 | 26 | /** 27 | * 异步刷盘 28 | */ 29 | ASYNC_FLUSH 30 | 31 | } 32 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/config/StoreConfigLoader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.config; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.utils.FileInfo; 22 | import com.wuba.wpaxos.utils.ThreadRenameFactory; 23 | 24 | import java.io.File; 25 | import java.io.IOException; 26 | import java.util.concurrent.Executors; 27 | import java.util.concurrent.ScheduledExecutorService; 28 | import java.util.concurrent.TimeUnit; 29 | 30 | /** 31 | * store config load dynamic 32 | */ 33 | public class StoreConfigLoader { 34 | private static StoreConfigLoader fileLoad = new StoreConfigLoader(); 35 | private String configPath; 36 | private FileInfo fileInfo = null; 37 | private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1, new ThreadRenameFactory("StoreConfigLoad Thread")); 38 | 39 | private StoreConfigLoader() {} 40 | 41 | public static StoreConfigLoader getInstance() { 42 | return fileLoad; 43 | } 44 | 45 | public void setFileInfo(String path) throws IOException { 46 | this.configPath = path; 47 | fileInfo = new FileInfo(new File(path)); 48 | } 49 | 50 | public String getConfigPath() { 51 | return configPath; 52 | } 53 | 54 | public void setConfigPath(String configPath) { 55 | this.configPath = configPath; 56 | } 57 | 58 | public void start(DynamicConfig dynamicConfig) { 59 | scheduler.scheduleWithFixedDelay(new ConfigObserverJob(fileInfo, dynamicConfig), 1, 1, TimeUnit.MINUTES); 60 | } 61 | } 62 | 63 | class ConfigObserverJob implements Runnable { 64 | private static final Logger logger = LogManager.getLogger(ConfigObserverJob.class); 65 | private static FileInfo fInfo; 66 | private static DynamicConfig dynamicConfig; 67 | 68 | public ConfigObserverJob(FileInfo fi, DynamicConfig dynamicConf) { 69 | fInfo = fi; 70 | dynamicConfig = dynamicConf; 71 | } 72 | 73 | @Override 74 | public void run() { 75 | try { 76 | if (fInfo != null && dynamicConfig != null) { 77 | File f = new File(fInfo.getFilePath()); 78 | if (f != null) { 79 | if (f.lastModified() != fInfo.getLastModifyTime()) { 80 | dynamicConfig.loadConfig(); 81 | fInfo.setLastModifyTime(f.lastModified()); 82 | logger.info("Store config is reload."); 83 | } 84 | } 85 | } 86 | } catch(Throwable th) { 87 | logger.error("", th); 88 | } 89 | } 90 | } -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/config/StorePathConfigHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.config; 17 | 18 | import java.io.File; 19 | 20 | /** 21 | * 存储文件路径管理 22 | */ 23 | public class StorePathConfigHelper { 24 | 25 | public static String getStorePathPhysicLog(final String rootDir) { 26 | return rootDir + File.separator + "physiclog"; 27 | } 28 | 29 | public static String getStoreCheckpoint(final String rootDir) { 30 | return rootDir + File.separator + "checkpoint"; 31 | } 32 | 33 | public static String getStorePathIndexDB(final String rootDir) { 34 | return rootDir + File.separator + "indexDB"; 35 | } 36 | 37 | public static String getStorePathVarStore(final String rootDir) { 38 | return rootDir + File.separator + "varlog"; 39 | } 40 | 41 | public static String getAbortFile(final String rootDir) { 42 | return rootDir + File.separator + "abort"; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/db/IndexDB.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.db; 17 | 18 | 19 | import com.wuba.wpaxos.config.WriteOptions; 20 | import com.wuba.wpaxos.store.FileID; 21 | 22 | /** 23 | * paxoslog索引 24 | */ 25 | public interface IndexDB { 26 | /** 27 | * 28 | * @param maxInstanceID 29 | */ 30 | void correctMaxInstanceID(long maxInstanceID); 31 | 32 | boolean putIndex(WriteOptions writeOptions, long instanceID, FileID value); 33 | 34 | FileID getIndex(long instanceID); 35 | 36 | void deleteOneIndex(long instanceId); 37 | 38 | boolean init() ; 39 | 40 | byte[] getMaxInstanceID(); 41 | 42 | void setMaxInstanceID(WriteOptions writeOptions, long instanceId); 43 | 44 | void setMastervariables(WriteOptions writeOptions, byte[] buffer); 45 | 46 | byte[] getMastervariables(); 47 | 48 | byte[] getSystemvariables(); 49 | 50 | void setSystemvariables(WriteOptions writeOptions, byte[] buffer); 51 | 52 | void setMinChosenInstanceID(WriteOptions writeOptions, long instanceId); 53 | 54 | byte[] getMinChosenInstanceID(); 55 | 56 | void destroy(); 57 | 58 | void deleteExpire(WriteOptions writeOptions, long maxInstanceId); 59 | } 60 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/pagecache/ReferenceResource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.wuba.wpaxos.store.pagecache; 15 | 16 | import java.util.concurrent.atomic.AtomicLong; 17 | 18 | /** 19 | * 引用计数基类,类似于C++智能指针实现 20 | */ 21 | public abstract class ReferenceResource { 22 | protected final AtomicLong refCount = new AtomicLong(1); 23 | protected volatile boolean available = true; 24 | protected volatile boolean cleanupOver = false; 25 | private volatile long firstShutdownTimestamp = 0; 26 | 27 | /** 28 | * 资源是否能HOLD住 29 | */ 30 | public synchronized boolean hold() { 31 | if (this.isAvailable()) { 32 | if (this.refCount.getAndIncrement() > 0) { 33 | return true; 34 | } else { 35 | this.refCount.getAndDecrement(); 36 | } 37 | } 38 | 39 | return false; 40 | } 41 | 42 | /** 43 | * 资源是否可用,即是否可被HOLD 44 | */ 45 | public boolean isAvailable() { 46 | return this.available; 47 | } 48 | 49 | /** 50 | * 禁止资源被访问 shutdown不允许调用多次,最好是由管理线程调用 51 | */ 52 | public void shutdown(final long intervalForcibly) { 53 | if (this.available) { 54 | this.available = false; 55 | this.firstShutdownTimestamp = System.currentTimeMillis(); 56 | this.release(); 57 | } 58 | // 强制shutdown 59 | else if (this.getRefCount() > 0) { 60 | if ((System.currentTimeMillis() - this.firstShutdownTimestamp) >= intervalForcibly) { 61 | this.refCount.set(-1000 - this.getRefCount()); 62 | this.release(); 63 | } 64 | } 65 | } 66 | 67 | public long getRefCount() { 68 | return this.refCount.get(); 69 | } 70 | 71 | /** 72 | * 释放资源 73 | */ 74 | public void release() { 75 | long value = this.refCount.decrementAndGet(); 76 | if (value > 0) { 77 | return; 78 | } 79 | 80 | synchronized (this) { 81 | // cleanup内部要对是否clean做处理 82 | this.cleanupOver = this.cleanup(value); 83 | } 84 | } 85 | 86 | public abstract boolean cleanup(final long currentRef); 87 | 88 | /** 89 | * 资源是否被清理完成 90 | */ 91 | public boolean isCleanupOver() { 92 | return this.refCount.get() <= 0 && this.cleanupOver; 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/service/CleanIndexDBService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.service; 17 | 18 | import java.util.List; 19 | 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | 23 | import com.wuba.wpaxos.config.WriteOptions; 24 | import com.wuba.wpaxos.store.DefaultDataBase; 25 | import com.wuba.wpaxos.store.DefaultLogStorage; 26 | import com.wuba.wpaxos.store.PhysicLog; 27 | import com.wuba.wpaxos.store.db.IndexDB; 28 | import com.wuba.wpaxos.utils.ServiceThread; 29 | 30 | /** 31 | * 定时删除 paxos log索引文件 32 | */ 33 | public class CleanIndexDBService extends ServiceThread { 34 | private static final Logger log = LogManager.getLogger(CleanIndexDBService.class); 35 | private DefaultLogStorage defaultLogStorage; 36 | 37 | public CleanIndexDBService(DefaultLogStorage defaultLogStorage) { 38 | this.defaultLogStorage = defaultLogStorage; 39 | } 40 | 41 | private void deleteExpiredFiles() { 42 | int maxDeleteIndexIdInterval = defaultLogStorage.getStoreConfig().getDeleteIndexIdInterval(); 43 | 44 | List dblist = this.defaultLogStorage.getDbList(); 45 | for (DefaultDataBase database : dblist) { 46 | if (!database.isAvailable()) { 47 | continue; 48 | } 49 | 50 | PhysicLog physicLog = database.getValueStore(); 51 | IndexDB indexDB = database.getIndexdb(); 52 | 53 | long minChosenInstanceID = database.getMinChosenInstanceId(); 54 | long instanceID = physicLog.getMinInstanceID() - 1; 55 | while (instanceID > minChosenInstanceID) { 56 | long maxDeleteInstanceID = Math.min(instanceID, minChosenInstanceID + maxDeleteIndexIdInterval); 57 | long deleteInterval = maxDeleteInstanceID - minChosenInstanceID; 58 | indexDB.deleteExpire(new WriteOptions(false), maxDeleteInstanceID); 59 | 60 | minChosenInstanceID = maxDeleteInstanceID; 61 | WriteOptions writeOptions = new WriteOptions(); 62 | writeOptions.setSync(true); 63 | int ret = this.defaultLogStorage.setMinChosenInstanceID(writeOptions, database.getMyGroupIdx(), maxDeleteInstanceID); 64 | if(ret != 0) { 65 | log.error("logStore setMinChosenInstanceID error, ret={}.", ret); 66 | return; 67 | } 68 | 69 | if (deleteInterval >= maxDeleteIndexIdInterval) { 70 | try { 71 | Thread.sleep(10); 72 | } 73 | catch (InterruptedException e) { 74 | log.error(e.getMessage(), e); 75 | } 76 | } 77 | } 78 | } 79 | } 80 | 81 | @Override 82 | public void run() { 83 | try { 84 | this.deleteExpiredFiles(); 85 | } catch (Exception e) { 86 | log.warn(this.getServiceName() + " service has exception. ", e); 87 | } 88 | } 89 | 90 | @Override 91 | public String getServiceName() { 92 | return CleanIndexDBService.class.getSimpleName(); 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/service/CleanIndexMappedFileService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.service; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.store.db.FileIndexDB; 22 | import com.wuba.wpaxos.utils.ServiceThread; 23 | 24 | /** 25 | * 一定会时间内不再访问的索引文件,取消内存映射 26 | */ 27 | public class CleanIndexMappedFileService extends ServiceThread { 28 | private static final Logger log = LogManager.getLogger(CleanIndexMappedFileService.class); 29 | private int groupId; 30 | private FileIndexDB fileIndexDB; 31 | 32 | public CleanIndexMappedFileService(int groupId, FileIndexDB fileIndexDB) { 33 | super(CleanIndexMappedFileService.class.getSimpleName() + "-" + groupId); 34 | this.fileIndexDB = fileIndexDB; 35 | this.groupId = groupId; 36 | } 37 | 38 | public void cleanUnMappedFiles() { 39 | fileIndexDB.getMapedFileQueue().cleanUnMappedFile(); 40 | } 41 | 42 | @Override 43 | public void run() { 44 | try { 45 | this.cleanUnMappedFiles(); 46 | } catch (Exception e) { 47 | log.warn(this.getServiceName() + " service has exception. ", e); 48 | } 49 | } 50 | 51 | @Override 52 | public String getServiceName() { 53 | return CleanIndexMappedFileService.class.getSimpleName() + "-" + groupId; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/service/CleanPhyMappedFileService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.service; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.store.DefaultDataBase; 22 | import com.wuba.wpaxos.store.DefaultLogStorage; 23 | import com.wuba.wpaxos.store.PhysicLog; 24 | import com.wuba.wpaxos.utils.ServiceThread; 25 | 26 | import java.util.List; 27 | 28 | /** 29 | * 一定会时间内不再访问的paxoslog文件,取消内存映射 30 | */ 31 | public class CleanPhyMappedFileService extends ServiceThread{ 32 | private static final Logger log = LogManager.getLogger(CleanPhyMappedFileService.class); 33 | private DefaultLogStorage fileLogStorage; 34 | 35 | public CleanPhyMappedFileService(DefaultLogStorage fileLogStorage) { 36 | this.fileLogStorage = fileLogStorage; 37 | } 38 | 39 | public void cleanUnMappedFiles() { 40 | List dblist = this.fileLogStorage.getDbList(); 41 | for (DefaultDataBase dataBase : dblist) { 42 | PhysicLog physicLog = dataBase.getValueStore(); 43 | if (physicLog != null) { 44 | physicLog.getMapedFileQueue().cleanUnMappedFile(); 45 | } 46 | } 47 | } 48 | 49 | @Override 50 | public void run() { 51 | try { 52 | this.cleanUnMappedFiles(); 53 | } catch (Exception e) { 54 | log.warn(this.getServiceName() + " service has exception. ", e); 55 | } 56 | } 57 | 58 | @Override 59 | public String getServiceName() { 60 | return CleanPhyMappedFileService.class.getSimpleName(); 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/store/service/CommitIndexService.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.store.service; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import com.wuba.wpaxos.store.DefaultLogStorage; 22 | import com.wuba.wpaxos.store.db.FileIndexDB; 23 | import com.wuba.wpaxos.utils.ServiceThread; 24 | 25 | /** 26 | * write index from writebuffer to filechannel 27 | */ 28 | public class CommitIndexService extends ServiceThread { 29 | private static final Logger log = LogManager.getLogger(CommitIndexService.class); 30 | private long lastCommitTimestamp = 0L; 31 | private static final int RetryTimesOver = 10; 32 | private int groupId; 33 | private FileIndexDB fileIndexDB; 34 | private DefaultLogStorage fileLogStorage; 35 | 36 | public CommitIndexService(DefaultLogStorage fileLogStorage, int groupId, FileIndexDB fileIndexDB) { 37 | super(CommitIndexService.class.getSimpleName() + "-" + groupId); 38 | this.fileLogStorage = fileLogStorage; 39 | this.groupId = groupId; 40 | this.fileIndexDB = fileIndexDB; 41 | } 42 | 43 | @Override 44 | public void run() { 45 | log.info(this.getServiceName() + " service started"); 46 | 47 | while (!this.isStopped()) { 48 | int interval = this.fileLogStorage.getStoreConfig().getCommitIntervalIndexdb(); 49 | 50 | int commitDataLeastPages = this.fileLogStorage.getStoreConfig().getCommitIndexDBLeastPages(); 51 | 52 | int commitDataThoroughInterval = this.fileLogStorage.getStoreConfig().getCommitIndexToroughInterval(); 53 | 54 | long begin = System.currentTimeMillis(); 55 | if (begin >= (this.lastCommitTimestamp + commitDataThoroughInterval)) { 56 | this.lastCommitTimestamp = begin; 57 | commitDataLeastPages = 0; 58 | } 59 | 60 | try { 61 | if (fileIndexDB != null) { 62 | boolean result = fileIndexDB.commit(commitDataLeastPages); 63 | long end = System.currentTimeMillis(); 64 | if (!result) { 65 | this.lastCommitTimestamp = end; 66 | //now wake up flush thread. 67 | this.fileIndexDB.getFlushIndexService().wakeup(); 68 | } 69 | 70 | if (end - begin > 2000) { 71 | log.info("Commit indexdb to file costs {} ms", end - begin); 72 | } 73 | this.waitForRunning(interval); 74 | } 75 | } catch (Throwable e) { 76 | log.warn(this.getServiceName() + " service has exception. ", e); 77 | } 78 | } 79 | 80 | if (fileIndexDB != null) { 81 | boolean result = false; 82 | for (int j = 0; j < RetryTimesOver && !result; j++) { 83 | result = fileIndexDB.commit(0); 84 | log.info(this.getServiceName() + " service shutdown, group : " + groupId + " retry " + (j + 1) + " times " + (result ? "OK" : "Not OK")); 85 | } 86 | } 87 | 88 | log.info(this.getServiceName() + " service end"); 89 | } 90 | 91 | @Override 92 | public String getServiceName() { 93 | return CommitIndexService.class.getSimpleName() + "-" + groupId; 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/storemachine/BatchSMCtx.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.storemachine; 17 | 18 | import java.util.ArrayList; 19 | import java.util.List; 20 | 21 | /** 22 | * Batch propose statemachine context 23 | */ 24 | public class BatchSMCtx { 25 | List smCtxList = new ArrayList(); 26 | 27 | public BatchSMCtx() { 28 | this.smCtxList = new ArrayList(); 29 | } 30 | 31 | public List getSmCtxList() { 32 | return smCtxList; 33 | } 34 | 35 | public void setSmCtxList(List smCtxList) { 36 | this.smCtxList = smCtxList; 37 | } 38 | 39 | public SMCtx getByIndex(int idx) { 40 | if (idx >= this.smCtxList.size()) { 41 | return null; 42 | } 43 | return this.smCtxList.get(idx); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/storemachine/InsideSM.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.storemachine; 17 | 18 | /** 19 | * 内置状态机 20 | */ 21 | public interface InsideSM extends StateMachine { 22 | 23 | public byte[] getCheckpointBuffer(); 24 | 25 | public UpdateCpRet updateByCheckpoint(byte[] cpBuffer); 26 | 27 | static class UpdateCpRet { 28 | private boolean isChange = false; 29 | private int ret; 30 | 31 | public UpdateCpRet () { 32 | } 33 | 34 | public UpdateCpRet(boolean isChanged, int ret) { 35 | this.isChange = isChanged; 36 | this.ret = ret; 37 | } 38 | 39 | public boolean isChange() { 40 | return isChange; 41 | } 42 | 43 | public void setChange(boolean isChange) { 44 | this.isChange = isChange; 45 | } 46 | 47 | public int getRet() { 48 | return ret; 49 | } 50 | 51 | public void setRet(int ret) { 52 | this.ret = ret; 53 | } 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/storemachine/SMCtx.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.storemachine; 17 | 18 | /** 19 | * state machine context 20 | */ 21 | public class SMCtx { 22 | private int smId; 23 | private Object pCtx; 24 | 25 | public SMCtx() { 26 | } 27 | 28 | public SMCtx(int smId, Object pCtx) { 29 | super(); 30 | this.smId = smId; 31 | this.pCtx = pCtx; 32 | } 33 | 34 | public int getSmId() { 35 | return smId; 36 | } 37 | 38 | public void setSmId(int smId) { 39 | this.smId = smId; 40 | } 41 | 42 | public Object getpCtx() { 43 | return pCtx; 44 | } 45 | 46 | public void setpCtx(Object pCtx) { 47 | this.pCtx = pCtx; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/ConfigManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import org.apache.logging.log4j.LogManager; 19 | import org.apache.logging.log4j.Logger; 20 | 21 | import java.io.IOException; 22 | 23 | public abstract class ConfigManager { 24 | private static final Logger logger = LogManager.getLogger(ConfigManager.class); 25 | 26 | public abstract String encode(); 27 | 28 | 29 | public abstract String encode(final boolean prettyFormat); 30 | 31 | 32 | public abstract void decode(final String jsonString); 33 | 34 | 35 | public abstract String configFilePath(); 36 | 37 | 38 | public boolean load() { 39 | String fileName = null; 40 | try { 41 | fileName = this.configFilePath(); 42 | String jsonString = MixAll.file2String(fileName); 43 | // 文件不存在,或者为空文件 44 | if (null == jsonString || jsonString.length() == 0) { 45 | return this.loadBak(); 46 | } 47 | else { 48 | this.decode(jsonString); 49 | logger.info("init {} OK", fileName); 50 | return true; 51 | } 52 | } 53 | catch (Exception e) { 54 | logger.error("init " + fileName + " Failed, and try to init backup file", e); 55 | return this.loadBak(); 56 | } 57 | } 58 | 59 | 60 | private boolean loadBak() { 61 | String fileName = null; 62 | try { 63 | fileName = this.configFilePath(); 64 | String jsonString = MixAll.file2String(fileName + ".bak"); 65 | if (jsonString != null && jsonString.length() > 0) { 66 | this.decode(jsonString); 67 | logger.info("init " + fileName + " OK"); 68 | return true; 69 | } 70 | } 71 | catch (Exception e) { 72 | logger.error("init " + fileName + " Failed", e); 73 | return false; 74 | } 75 | 76 | return true; 77 | } 78 | 79 | 80 | public synchronized void persist() { 81 | String jsonString = this.encode(true); 82 | if (jsonString != null) { 83 | String fileName = this.configFilePath(); 84 | try { 85 | MixAll.string2File(jsonString, fileName); 86 | } 87 | catch (IOException e) { 88 | logger.error("persist file Exception, " + fileName, e); 89 | } 90 | } 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/Crc32.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.zip.CRC32; 19 | 20 | public class Crc32 { 21 | public static final int crc32(byte[] array) { 22 | if (array != null) { 23 | return crc32(array, 0, array.length); 24 | } 25 | 26 | return 0; 27 | } 28 | 29 | public static final int crc32(byte[] array, int offset, int length) { 30 | CRC32 crc32 = new CRC32(); 31 | crc32.update(array, offset, length); 32 | return (int) (crc32.getValue() & 0x7FFFFFFF); 33 | } 34 | 35 | public static final int crc32(int crc, byte[] array, int length, int offset) { 36 | CRC32 crc32 = new CRC32(); 37 | crc32.update(crc); 38 | crc32.update(array, offset, length - offset); 39 | return (int) (crc32.getValue() & 0x7FFFFFFF); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/FileInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.io.File; 19 | import java.io.IOException; 20 | 21 | public class FileInfo { 22 | 23 | private String fileName; 24 | 25 | private String filePath; 26 | 27 | private long fileSize; 28 | 29 | private long lastModifyTime; 30 | 31 | public FileInfo(File f) throws IOException{ 32 | this.fileName = f.getName(); 33 | this.filePath = f.getCanonicalPath(); 34 | this.fileSize = f.length(); 35 | this.lastModifyTime = f.lastModified(); 36 | } 37 | 38 | public String getFileName() { 39 | return fileName; 40 | } 41 | 42 | public void setFileName(String fileName) { 43 | this.fileName = fileName; 44 | } 45 | 46 | public String getFilePath() { 47 | return filePath; 48 | } 49 | 50 | public void setFilePath(String filePath) { 51 | this.filePath = filePath; 52 | } 53 | 54 | public long getFileSize() { 55 | return fileSize; 56 | } 57 | 58 | public void setFileSize(long fileSize) { 59 | this.fileSize = fileSize; 60 | } 61 | 62 | public long getLastModifyTime() { 63 | return lastModifyTime; 64 | } 65 | 66 | public void setLastModifyTime(long lastModifyTime) { 67 | this.lastModifyTime = lastModifyTime; 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/FileUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.io.File; 19 | import java.util.ArrayList; 20 | import java.util.List; 21 | 22 | public class FileUtils { 23 | 24 | public static boolean isDir(String path){ 25 | File file = new File(path); 26 | return file.isDirectory(); 27 | } 28 | 29 | public static void deleteDir(String dirPath) { 30 | File dir = new File(dirPath); 31 | if(!dir.isDirectory()) return ; 32 | 33 | File[] files = dir.listFiles(); 34 | if(files != null && files.length > 0) { 35 | for(File file : files) { 36 | 37 | if(".".equals(file.getName()) || "..".equals(file.getName())) { 38 | continue ; 39 | } 40 | 41 | if(file.isDirectory()) { 42 | deleteDir(file.getAbsolutePath()); 43 | } else { 44 | file.delete(); 45 | } 46 | } 47 | } 48 | 49 | dir.delete(); 50 | } 51 | 52 | public static List iterDir(String dirPath) { 53 | List filePathList = new ArrayList(); 54 | File dir = new File(dirPath); 55 | if(!dir.isDirectory()) { 56 | return filePathList; 57 | } 58 | 59 | iterDir(dir, filePathList); 60 | return filePathList; 61 | } 62 | 63 | private static void iterDir(File dir, List filePathList) { 64 | File[] files = dir.listFiles(); 65 | if(files != null && files.length > 0) { 66 | for(File file : files) { 67 | if(file.isDirectory()) { 68 | iterDir(file, filePathList); 69 | } else { 70 | filePathList.add(file.getName()); 71 | } 72 | } 73 | } 74 | } 75 | 76 | } 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/JavaOriTypeWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | public class JavaOriTypeWrapper { 19 | 20 | private T value; 21 | 22 | public JavaOriTypeWrapper() { 23 | } 24 | 25 | public JavaOriTypeWrapper(T value) { 26 | this.value = value; 27 | } 28 | 29 | public T getValue() { 30 | return value; 31 | } 32 | 33 | public void setValue(T value) { 34 | this.value = value; 35 | } 36 | 37 | 38 | 39 | } 40 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/LibC.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.wuba.wpaxos.utils; 15 | 16 | import com.sun.jna.Library; 17 | import com.sun.jna.Native; 18 | import com.sun.jna.NativeLong; 19 | import com.sun.jna.Platform; 20 | import com.sun.jna.Pointer; 21 | 22 | public interface LibC extends Library { 23 | @SuppressWarnings("deprecation") 24 | LibC INSTANCE = (LibC) Native.loadLibrary(Platform.isWindows() ? "msvcrt" : "c", LibC.class); 25 | 26 | int MADV_WILLNEED = 3; 27 | int MADV_DONTNEED = 4; 28 | 29 | int MCL_CURRENT = 1; 30 | int MCL_FUTURE = 2; 31 | int MCL_ONFAULT = 4; 32 | 33 | /* sync memory asynchronously */ 34 | int MS_ASYNC = 0x0001; 35 | /* invalidate mappings & caches */ 36 | int MS_INVALIDATE = 0x0002; 37 | /* synchronous memory sync */ 38 | int MS_SYNC = 0x0004; 39 | 40 | int mlock(Pointer var1, NativeLong var2); 41 | 42 | int munlock(Pointer var1, NativeLong var2); 43 | 44 | int madvise(Pointer var1, NativeLong var2, int var3); 45 | 46 | Pointer memset(Pointer p, int v, long len); 47 | 48 | int mlockall(int flags); 49 | 50 | int msync(Pointer p, NativeLong length, int flags); 51 | } 52 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/MasterGroupStatPrinter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.concurrent.ConcurrentHashMap; 19 | 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | 23 | /** 24 | * 统计并输出master group 25 | * @author Service Platform Architecture Team (spat@58.com) 26 | * 27 | */ 28 | public class MasterGroupStatPrinter { 29 | 30 | private static final Logger logger = LogManager.getLogger(MasterGroupStatPrinter.class); 31 | 32 | private static volatile ConcurrentHashMap masterGroupSet = new ConcurrentHashMap(); 33 | 34 | static { 35 | Thread t = new Thread(new PrintTask()); 36 | t.setName("MasterGroupStatPrinter-PrintTask"); 37 | t.start(); 38 | } 39 | 40 | public static void put(Integer groupId) { 41 | masterGroupSet.put(groupId, new Object()); 42 | } 43 | 44 | public static class PrintTask implements Runnable { 45 | 46 | @Override 47 | public void run() { 48 | while(true) { 49 | try { 50 | ConcurrentHashMap tmpSet = masterGroupSet; 51 | masterGroupSet = new ConcurrentHashMap(); 52 | 53 | logger.info("master group " + tmpSet.keySet()); 54 | } catch (Throwable e) { 55 | logger.error("PrintTask throws exception", e); 56 | } finally { 57 | Time.sleep(15000); 58 | } 59 | } 60 | } 61 | } 62 | 63 | } 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/Notifier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.io.IOException; 19 | 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | 23 | import com.wuba.wpaxos.helper.SerialLock; 24 | 25 | public class Notifier { 26 | private final Logger logger = LogManager.getLogger(Notifier.class); 27 | private long id; 28 | private int ret = -1; 29 | 30 | private int commitRet = -1; 31 | private boolean isCommitEnd = false; 32 | private SerialLock serialLock = new SerialLock(); 33 | 34 | public void init() { 35 | this.id = Thread.currentThread().getId(); 36 | serialLock.lock(); 37 | this.commitRet = -1; 38 | this.isCommitEnd = false; 39 | serialLock.unLock(); 40 | } 41 | 42 | public int waitNotify() { 43 | this.serialLock.lock(); 44 | try { 45 | while (!isCommitEnd) { 46 | this.serialLock.waitTime(1000); 47 | } 48 | } catch(Exception e) { 49 | logger.error("", e); 50 | } finally { 51 | this.serialLock.unLock(); 52 | } 53 | return this.commitRet; 54 | } 55 | 56 | public void sendNotify(int ret) throws IOException { 57 | serialLock.lock(); 58 | this.commitRet = ret; 59 | this.isCommitEnd = true; 60 | serialLock.interupt(); 61 | serialLock.unLock(); 62 | } 63 | 64 | public int getRet() { 65 | return ret; 66 | } 67 | 68 | public long getId() { 69 | return id; 70 | } 71 | 72 | public void setId(long id) { 73 | this.id = id; 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/NotifierPool.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.concurrent.ConcurrentHashMap; 19 | 20 | public class NotifierPool { 21 | 22 | private ConcurrentHashMap notifierMap = new ConcurrentHashMap(); 23 | 24 | public Notifier getNotifier(Long id) { 25 | Notifier notifier = notifierMap.get(id); 26 | if(notifier != null) { 27 | return notifier; 28 | } 29 | 30 | synchronized (notifierMap) { 31 | notifier = notifierMap.get(id); 32 | if(notifier == null) { 33 | notifier = new Notifier(); 34 | notifierMap.put(id, notifier); 35 | } 36 | } 37 | return notifier; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/OtherUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.Random; 19 | 20 | public class OtherUtils { 21 | public static Random random = new Random(99999); 22 | 23 | public static long genGid(long nodeID) { 24 | return (nodeID ^ fastRand()) + fastRand(); 25 | } 26 | 27 | public static int fastRand() { 28 | return random.nextInt(10000); 29 | } 30 | 31 | public static long getSystemMS() { 32 | return System.currentTimeMillis(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/ThreadFactoryImpl.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.concurrent.ThreadFactory; 19 | import java.util.concurrent.atomic.AtomicLong; 20 | 21 | public class ThreadFactoryImpl implements ThreadFactory{ 22 | private final AtomicLong threadIndex = new AtomicLong(0); 23 | private final String threadNamePrefix; 24 | 25 | public ThreadFactoryImpl(final String threadNamePrefix) { 26 | this.threadNamePrefix = threadNamePrefix; 27 | } 28 | 29 | @Override 30 | public Thread newThread(Runnable r) { 31 | return new Thread(r, threadNamePrefix + this.threadIndex.incrementAndGet()); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/ThreadRenameFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | import java.util.concurrent.ThreadFactory; 19 | import java.util.concurrent.atomic.AtomicInteger; 20 | 21 | public class ThreadRenameFactory implements ThreadFactory { 22 | static final AtomicInteger poolNumber = new AtomicInteger(1); 23 | final ThreadGroup group; 24 | final AtomicInteger threadNumber = new AtomicInteger(1); 25 | final String namePrefix; 26 | 27 | public ThreadRenameFactory(String threadNamePrefix) { 28 | SecurityManager s = System.getSecurityManager(); 29 | group = (s != null) ? s.getThreadGroup() : Thread.currentThread() 30 | .getThreadGroup(); 31 | namePrefix = threadNamePrefix + "-pool-" + poolNumber.getAndIncrement() 32 | + "-tid-"; 33 | } 34 | 35 | @Override 36 | public Thread newThread(Runnable r) { 37 | Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0); 38 | if (t.isDaemon()) { 39 | t.setDaemon(false); 40 | } 41 | if (t.getPriority() != Thread.NORM_PRIORITY) { 42 | t.setPriority(Thread.NORM_PRIORITY); 43 | } 44 | return t; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/Time.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed under the Apache License, Version 2.0 (the "License"); 3 | * you may not use this file except in compliance with the License. 4 | * You may obtain a copy of the License at 5 | * 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | package com.wuba.wpaxos.utils; 15 | 16 | public class Time { 17 | 18 | /** 19 | * 用于获取时钟时间 20 | * @return 21 | */ 22 | public static long getTimestampMS() { 23 | return System.currentTimeMillis(); 24 | } 25 | 26 | /** 27 | * 用于时间统计 28 | * @return 29 | */ 30 | public static long getSteadyClockMS() { 31 | //TODO CHECK 32 | //return System.nanoTime() / 10000000; 33 | return System.currentTimeMillis(); 34 | } 35 | 36 | /** 37 | * 38 | * @param timeMs 毫秒 39 | * @throws InterruptedException 40 | */ 41 | public static void sleep(int timeMs) { 42 | try { 43 | Thread.sleep(timeMs); 44 | } catch (InterruptedException e) { 45 | e.printStackTrace(); 46 | } 47 | } 48 | 49 | } 50 | -------------------------------------------------------------------------------- /core/src/main/java/com/wuba/wpaxos/utils/TimeStat.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.utils; 17 | 18 | public class TimeStat { 19 | private long timeStmp; 20 | 21 | public TimeStat() { 22 | this.timeStmp = Time.getSteadyClockMS(); 23 | } 24 | 25 | public int point() { 26 | long nowTime = Time.getSteadyClockMS(); 27 | int passTime = 0; 28 | if (nowTime > this.timeStmp) { 29 | passTime = (int) (nowTime - this.timeStmp); 30 | } 31 | 32 | this.timeStmp = nowTime; 33 | return passTime; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /img/clean.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/img/clean.png -------------------------------------------------------------------------------- /img/framework.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/img/framework.png -------------------------------------------------------------------------------- /img/store.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/img/store.png -------------------------------------------------------------------------------- /img/structure.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/img/structure.png -------------------------------------------------------------------------------- /img/wpaxos-wechat.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wuba/WPaxos/1388a234c540a4f2b6e15499a98816cb243d7beb/img/wpaxos-wechat.png -------------------------------------------------------------------------------- /sample/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 6 | wpaxos 7 | io.github.wuba 8 | 1.0.1 9 | 10 | 4.0.0 11 | 12 | wpaxos.sample 13 | 14 | 15 | ${project.groupId} 16 | wpaxos.core 17 | ${project.version} 18 | 19 | 20 | org.rocksdb 21 | rocksdbjni 22 | 23 | 24 | 25 | 26 | 27 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/echo/EchoClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.echo; 17 | 18 | import com.wuba.wpaxos.comm.NodeInfo; 19 | import com.wuba.wpaxos.sample.util.NodeUtil; 20 | import org.apache.logging.log4j.core.config.ConfigurationSource; 21 | import org.apache.logging.log4j.core.config.Configurator; 22 | 23 | import java.io.BufferedReader; 24 | import java.io.File; 25 | import java.io.FileInputStream; 26 | import java.io.InputStreamReader; 27 | import java.util.List; 28 | 29 | public class EchoClient { 30 | 31 | public static void main(String[] args) throws Exception { 32 | if (args.length != 5) { 33 | System.out.println("arguments num is wrong , they are[rootPath,myNode,nodeList,groupCount,indexType]"); 34 | System.exit(1); 35 | } 36 | 37 | String rootPath = args[0]; 38 | String log4jConfig = rootPath + File.separator + "conf" + File.separator + "log4j.properties"; 39 | ConfigurationSource src = new ConfigurationSource(new FileInputStream(log4jConfig)); 40 | Configurator.initialize(EchoClient.class.getClassLoader(), src); 41 | NodeInfo myNode = NodeUtil.parseIpPort(args[1]); 42 | List nodeInfoList = NodeUtil.parseIpPortList(args[2]); 43 | int groupCount = Integer.parseInt(args[3]); 44 | int indexType = Integer.parseInt(args[4]); 45 | 46 | EchoServer echoServer = new EchoServer(myNode, nodeInfoList, groupCount, rootPath, indexType); 47 | 48 | echoServer.runPaxos(); 49 | System.out.println("echo server start, ip [" + myNode.getIp() + "] port [" + myNode.getPort() + "]"); 50 | 51 | BufferedReader br = new BufferedReader(new InputStreamReader(System.in)); 52 | int idx = 0; 53 | while(true) { 54 | System.out.println("please input : "); 55 | String echoReqValue = br.readLine(); 56 | 57 | if("quit".equals(echoReqValue)) break; 58 | 59 | try { 60 | if("1".equals(echoReqValue)) { 61 | for(int i = 0; i < 50; i++) { 62 | String echoRespValue = echoServer.echo("wpaxos_test" + idx, 0); 63 | System.out.println("echo response : " + echoRespValue); 64 | idx++; 65 | Thread.sleep(1000); 66 | } 67 | } else { 68 | echoServer.addMember(NodeUtil.parseIpPort(new String(echoReqValue))); 69 | System.out.println(echoServer.getAllMembers()); 70 | } 71 | } catch(Exception e) { 72 | e.printStackTrace(); 73 | } 74 | } 75 | 76 | br.close(); 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/echo/EchoSM.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.echo; 17 | 18 | import com.wuba.wpaxos.storemachine.SMCtx; 19 | import com.wuba.wpaxos.storemachine.StateMachine; 20 | import com.wuba.wpaxos.utils.JavaOriTypeWrapper; 21 | 22 | import java.util.List; 23 | 24 | public class EchoSM implements StateMachine { 25 | public static final int SMID = 1; 26 | private int groupId; 27 | 28 | public EchoSM(int i) { 29 | this.groupId = i; 30 | } 31 | 32 | @Override 33 | public int getSMID() { 34 | return SMID; 35 | } 36 | 37 | @Override 38 | public boolean execute(int groupIdx, long instanceID, byte[] paxosValue, SMCtx smCtx) { 39 | System.out.println("[SM Execute] ok, smid " + this.getSMID() + " instanceid " + instanceID + " value " + new String(paxosValue)); 40 | 41 | //only commiter node have SMCtx. 42 | if(smCtx != null && smCtx.getpCtx() != null) { 43 | EchoSMCtx peCtx = (EchoSMCtx)smCtx.getpCtx(); 44 | peCtx.setExecuteRet(0); 45 | peCtx.setEchoRespValue(paxosValue); 46 | } 47 | 48 | executeForCheckpoint(groupIdx, instanceID, paxosValue); 49 | return true; 50 | } 51 | 52 | @Override 53 | public boolean executeForCheckpoint(int groupIdx, long instanceID, byte[] paxosValue) { 54 | // TODO Auto-generated method stub 55 | return false; 56 | } 57 | 58 | @Override 59 | public long getCheckpointInstanceID(int groupIdx) { 60 | // TODO Auto-generated method stub 61 | return 0; 62 | } 63 | 64 | @Override 65 | public int lockCheckpointState() { 66 | // TODO Auto-generated method stub 67 | return 0; 68 | } 69 | 70 | @Override 71 | public int getCheckpointState(int groupIdx, JavaOriTypeWrapper dirPath, List fileList) { 72 | // TODO Auto-generated method stub 73 | return 0; 74 | } 75 | 76 | @Override 77 | public void unLockCheckpointState() { 78 | // TODO Auto-generated method stub 79 | 80 | } 81 | 82 | @Override 83 | public int loadCheckpointState(int groupIdx, String checkpointTmpFileDirPath, List fileList, 84 | long checkpointInstanceID) { 85 | // TODO Auto-generated method stub 86 | return 0; 87 | } 88 | 89 | @Override 90 | public byte[] beforePropose(int groupIdx, byte[] sValue) { 91 | // TODO Auto-generated method stub 92 | return null; 93 | } 94 | 95 | @Override 96 | public boolean needCallBeforePropose() { 97 | // TODO Auto-generated method stub 98 | return false; 99 | } 100 | 101 | @Override 102 | public void fixCheckpointByMinChosenInstanceId(long minChosenInstanceID) { 103 | // TODO Auto-generated method stub 104 | 105 | } 106 | 107 | public int getGroupId() { 108 | return groupId; 109 | } 110 | 111 | public void setGroupId(int groupId) { 112 | this.groupId = groupId; 113 | } 114 | } 115 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/echo/EchoSMCtx.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.echo; 17 | 18 | public class EchoSMCtx { 19 | public int executeRet; 20 | private byte[] echoRespValue; 21 | 22 | public EchoSMCtx() { 23 | executeRet = -1; 24 | } 25 | 26 | public int getExecuteRet() { 27 | return executeRet; 28 | } 29 | 30 | public void setExecuteRet(int executeRet) { 31 | this.executeRet = executeRet; 32 | } 33 | 34 | public byte[] getEchoRespValue() { 35 | return echoRespValue; 36 | } 37 | 38 | public void setEchoRespValue(byte[] echoRespValue) { 39 | this.echoRespValue = echoRespValue; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/echo/start.sh: -------------------------------------------------------------------------------- 1 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30000 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 0 2 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30001 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 0 3 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30002 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 0 4 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/election/ElectionMain.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.election; 17 | 18 | import com.wuba.wpaxos.comm.NodeInfo; 19 | import com.wuba.wpaxos.sample.echo.EchoClient; 20 | import com.wuba.wpaxos.sample.util.NodeUtil; 21 | import org.apache.logging.log4j.core.config.ConfigurationSource; 22 | import org.apache.logging.log4j.core.config.Configurator; 23 | 24 | import java.io.File; 25 | import java.io.FileInputStream; 26 | import java.util.HashSet; 27 | import java.util.List; 28 | import java.util.Set; 29 | 30 | public class ElectionMain { 31 | 32 | public static void main(String[] args) throws Exception { 33 | if (args.length != 4) { 34 | System.out.println("arguments num is wrong , they are[rootPath,myNode,nodeList,groupCount]"); 35 | System.exit(1); 36 | } 37 | 38 | String rootPath = args[0]; 39 | String log4jConfig = rootPath + File.separator + "conf" + File.separator + "log4j.properties"; 40 | ConfigurationSource src = new ConfigurationSource(new FileInputStream(log4jConfig)); 41 | Configurator.initialize(EchoClient.class.getClassLoader(), src); 42 | NodeInfo myNode = NodeUtil.parseIpPort(args[1]); 43 | List nodeInfoList = NodeUtil.parseIpPortList(args[2]); 44 | int groupCount = Integer.parseInt(args[3]); 45 | 46 | ElectionServer electionServer = new ElectionServer(myNode, nodeInfoList, groupCount, rootPath); 47 | 48 | electionServer.runPaxos(); 49 | System.out.println("election server start, ip [" + myNode.getIp() + "] port [" + myNode.getPort() + "]"); 50 | 51 | while(true) { 52 | Set masterSet = new HashSet(); 53 | for (int i = 0; i < groupCount; i++) { 54 | if (electionServer.isMaster(i)) { 55 | masterSet.add(i); 56 | } 57 | } 58 | 59 | System.out.println("I'm master groups of " + masterSet); 60 | 61 | Thread.sleep(10*1000); 62 | } 63 | } 64 | } 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/election/start.sh: -------------------------------------------------------------------------------- 1 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30000 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 2 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30001 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 3 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30002 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 1 4 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/ClientRole.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | public class ClientRole { 19 | private static final int master = 0; 20 | private static final int slave = 1; 21 | private static final int leaner = 2; 22 | public static int role = 0; 23 | 24 | public static boolean isLeaner() { 25 | return role == leaner; 26 | } 27 | 28 | public static void setLeaner() { 29 | ClientRole.role = leaner; 30 | } 31 | 32 | public static boolean isMaster() { 33 | return role == master; 34 | } 35 | 36 | public static boolean isSlave() { 37 | return role == slave; 38 | } 39 | 40 | 41 | } 42 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVClientLeaner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | import com.wuba.wpaxos.comm.NodeInfo; 19 | import com.wuba.wpaxos.sample.util.NodeUtil; 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | import org.apache.logging.log4j.core.config.ConfigurationSource; 23 | import org.apache.logging.log4j.core.config.Configurator; 24 | 25 | import java.io.File; 26 | import java.io.FileInputStream; 27 | import java.util.List; 28 | 29 | public class KVClientLeaner { 30 | private static Logger logger; 31 | 32 | public static void main(String[] args) throws Exception { 33 | if (args.length != 5) { 34 | System.out.println("arguments num is wrong , they are[rootPath,myNode,nodeList,groupCount,indexType]"); 35 | System.exit(1); 36 | } 37 | ClientRole.setLeaner(); 38 | String rootPath = args[0]; 39 | String log4jConfig = rootPath + File.separator + "conf" + File.separator + "log4j.properties"; 40 | ConfigurationSource src = new ConfigurationSource(new FileInputStream(log4jConfig)); 41 | Configurator.initialize(KVClientLeaner.class.getClassLoader(), src); 42 | logger = LogManager.getLogger(KVClientLeaner.class); 43 | NodeInfo myNode = NodeUtil.parseIpPort(args[1]); 44 | List nodeInfoList = NodeUtil.parseIpPortList(args[2]); 45 | final int groupCount = Integer.parseInt(args[3]); 46 | final int indexType = Integer.parseInt(args[4]); 47 | final KVTestServer fileServer = new KVTestServer(myNode, nodeInfoList, groupCount, rootPath, indexType); 48 | fileServer.runPaxos(); 49 | logger.info("paxos leaner run success!"); 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVClientSlave.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | import com.wuba.wpaxos.comm.NodeInfo; 19 | import com.wuba.wpaxos.sample.util.NodeUtil; 20 | import org.apache.logging.log4j.LogManager; 21 | import org.apache.logging.log4j.Logger; 22 | import org.apache.logging.log4j.core.config.ConfigurationSource; 23 | import org.apache.logging.log4j.core.config.Configurator; 24 | 25 | import java.io.File; 26 | import java.io.FileInputStream; 27 | import java.util.List; 28 | 29 | public class KVClientSlave { 30 | private static Logger logger; 31 | 32 | public static void main(String[] args) throws Exception { 33 | if (args.length != 5) { 34 | System.out.println("arguments num is wrong , they are[rootPath,myNode,nodeList,groupCount,indexType]"); 35 | System.exit(1); 36 | } 37 | String rootPath = args[0]; 38 | String log4jConfig = rootPath + File.separator + "conf" + File.separator + "log4j.properties"; 39 | ConfigurationSource src = new ConfigurationSource(new FileInputStream(log4jConfig)); 40 | Configurator.initialize(KVClientSlave.class.getClassLoader(), src); 41 | logger = LogManager.getLogger(KVClientSlave.class); 42 | NodeInfo myNode = NodeUtil.parseIpPort(args[1]); 43 | List nodeInfoList = NodeUtil.parseIpPortList(args[2]); 44 | final int groupCount = Integer.parseInt(args[3]); 45 | final int indexType = Integer.parseInt(args[4]); 46 | final KVTestServer fileServer = new KVTestServer(myNode, nodeInfoList, groupCount, rootPath, indexType); 47 | fileServer.runPaxos(); 48 | logger.info("paxos slave run success!"); 49 | } 50 | 51 | } 52 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVOPValue.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | import com.alibaba.fastjson.JSON; 19 | 20 | public class KVOPValue { 21 | private String key; 22 | private String value; 23 | private int op; 24 | 25 | public KVOPValue() { 26 | } 27 | 28 | public KVOPValue(String key, String value, int op) { 29 | this.key = key; 30 | this.value = value; 31 | this.op = op; 32 | } 33 | 34 | public KVOPValue(String key, int op) { 35 | this.key = key; 36 | this.op = op; 37 | } 38 | 39 | public String getKey() { 40 | return key; 41 | } 42 | 43 | public void setKey(String key) { 44 | this.key = key; 45 | } 46 | 47 | public String getValue() { 48 | return value; 49 | } 50 | 51 | public void setValue(String value) { 52 | this.value = value; 53 | } 54 | 55 | public int getOp() { 56 | return op; 57 | } 58 | 59 | public void setOp(int op) { 60 | this.op = op; 61 | } 62 | 63 | public byte[] toBytes() { 64 | return JSON.toJSONBytes(this); 65 | } 66 | 67 | public static KVOPValue fromBytes(byte[] bs){ 68 | return JSON.parseObject(bs, KVOPValue.class); 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVOperation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | public enum KVOperation { 19 | WRITE(0,"写"),READ(1,"读"),DELETE(2,"删除"); 20 | private int op; 21 | private String des; 22 | 23 | KVOperation(int op, String des) { 24 | this.op = op; 25 | this.des = des; 26 | } 27 | 28 | public int getOp() { 29 | return op; 30 | } 31 | 32 | public String getDes() { 33 | return des; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | public class KVResult { 19 | public static final int SUCCESS = 0; 20 | public static final int KEY_EXIST = 1; 21 | public static final int KEY_NOT_EXIST = 2; 22 | public static final int PAXOS_ERROR = 3; 23 | public static final int ROCKSDB_ERROR = 4; 24 | } 25 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/KVTestResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv; 17 | 18 | public class KVTestResult { 19 | private int res; 20 | private String value; 21 | 22 | public KVTestResult() { 23 | } 24 | 25 | public KVTestResult(int res) { 26 | this.res = res; 27 | } 28 | 29 | public int getRes() { 30 | return res; 31 | } 32 | 33 | public void setRes(int res) { 34 | this.res = res; 35 | } 36 | 37 | public String getValue() { 38 | return value; 39 | } 40 | 41 | public void setValue(String value) { 42 | this.value = value; 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/rocksdb/RocksDbConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.kv.rocksdb; 17 | 18 | import org.rocksdb.CompactionStyle; 19 | import org.rocksdb.CompressionType; 20 | 21 | public class RocksDbConfig { 22 | public static final boolean createIfMissing = true; 23 | public static final int maxOpenFile = -1; 24 | public static final long writeBufferSize = 67108864; 25 | public static final int maxWriteBufferNumber = 16; 26 | public static final int writeBufferNumberToMerge = 1; 27 | public static final int levelZeroFileNumCompactionTrigger = 10; 28 | public static final int level0SlowdownWritesTrigger = 20; 29 | public static final int level0StopWritesTrigger = 40; 30 | public static final int maxBackgroundCompactions = 10; 31 | public static final int maxBackgroundFlushes = 1; 32 | public static final double memtablePrefixBloomSizeRatio = 0.125; 33 | public static final int boolmFilter = 10; 34 | public static final CompressionType compressionType = CompressionType.NO_COMPRESSION; 35 | public static final CompactionStyle compactionStyle = CompactionStyle.LEVEL; 36 | public static final boolean useFsync = false; 37 | public static final long targetFileSizeBase = 12582912; 38 | public static final long maxFileLevelBase = 10485760; 39 | public static final long maxLogFileSize = 5368709120L; 40 | public static final int maxBackgroundJob = 10; 41 | } 42 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/kv/start.sh: -------------------------------------------------------------------------------- 1 | master:nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30000 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 true 10 50 5 20 0 2 | slave:nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30001 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 0 3 | slave:nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30002 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 0 4 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/simple/SimpleSM.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.simple; 17 | 18 | import com.wuba.wpaxos.storemachine.SMCtx; 19 | import com.wuba.wpaxos.storemachine.StateMachine; 20 | import com.wuba.wpaxos.utils.JavaOriTypeWrapper; 21 | 22 | import java.util.List; 23 | 24 | public class SimpleSM implements StateMachine { 25 | private int groupId; 26 | private long checkPointInstanceId; 27 | 28 | public SimpleSM(int group) { 29 | this.groupId = group; 30 | } 31 | 32 | @Override 33 | public int getSMID() { 34 | return 3; 35 | } 36 | 37 | @Override 38 | public boolean execute(int groupIdx, long instanceID, byte[] paxosValue, SMCtx smCtx) { 39 | // logger.info("simple sm execute."); 40 | executeForCheckpoint(groupIdx, instanceID, paxosValue); 41 | return true; 42 | } 43 | 44 | @Override 45 | public boolean executeForCheckpoint(int groupIdx, long instanceID, 46 | byte[] paxosValue) { 47 | // TODO Auto-generated method stub 48 | this.checkPointInstanceId = instanceID; 49 | return true; 50 | } 51 | 52 | @Override 53 | public long getCheckpointInstanceID(int groupIdx) { 54 | // TODO Auto-generated method stub 55 | return checkPointInstanceId; 56 | } 57 | 58 | @Override 59 | public int lockCheckpointState() { 60 | // TODO Auto-generated method stub 61 | return 0; 62 | } 63 | 64 | @Override 65 | public int getCheckpointState(int groupIdx, 66 | JavaOriTypeWrapper dirPath, List fileList) { 67 | // TODO Auto-generated method stub 68 | return 0; 69 | } 70 | 71 | @Override 72 | public void unLockCheckpointState() { 73 | // TODO Auto-generated method stub 74 | } 75 | 76 | @Override 77 | public int loadCheckpointState(int groupIdx, 78 | String checkpointTmpFileDirPath, List fileList, 79 | long checkpointInstanceID) { 80 | // TODO Auto-generated method stub 81 | return 0; 82 | } 83 | 84 | @Override 85 | public byte[] beforePropose(int groupIdx, byte[] sValue) { 86 | // TODO Auto-generated method stub 87 | return null; 88 | } 89 | 90 | @Override 91 | public boolean needCallBeforePropose() { 92 | // TODO Auto-generated method stub 93 | return false; 94 | } 95 | 96 | @Override 97 | public void fixCheckpointByMinChosenInstanceId(long minChosenInstanceID) { 98 | // TODO Auto-generated method stub 99 | if (minChosenInstanceID > checkPointInstanceId) { 100 | checkPointInstanceId = minChosenInstanceID; 101 | } 102 | } 103 | 104 | public int getGroupId() { 105 | return groupId; 106 | } 107 | 108 | public void setGroupId(int groupId) { 109 | this.groupId = groupId; 110 | } 111 | } 112 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/simple/start.sh: -------------------------------------------------------------------------------- 1 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30000 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 true 10 50 5 20 0 512 2 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30001 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 true 10 50 5 20 0 512 3 | nohup java -jar jar.jar /opt/wpaxos/ 127.0.0.1:30002 127.0.0.1:30000,127.0.0.1:30001,127.0.0.1:30002 8 true 10 50 5 20 0 512 4 | -------------------------------------------------------------------------------- /sample/src/main/java/com/wuba/wpaxos/sample/util/NodeUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2005-present, 58.com. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package com.wuba.wpaxos.sample.util; 17 | 18 | import com.wuba.wpaxos.comm.NodeInfo; 19 | 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | 23 | public class NodeUtil { 24 | public static NodeInfo parseIpPort(String ipPort) { 25 | NodeInfo nodeInfo = new NodeInfo(); 26 | String[] arr = ipPort.split(":"); 27 | nodeInfo.setIp(arr[0]); 28 | nodeInfo.setPort(Integer.parseInt(arr[1])); 29 | return nodeInfo; 30 | } 31 | 32 | public static List parseIpPortList(String ipPorts) { 33 | List ipPortList = new ArrayList(); 34 | 35 | String[] ipPortArr = ipPorts.split(","); 36 | for (String ipPort : ipPortArr) { 37 | NodeInfo nodeInfo = parseIpPort(ipPort); 38 | ipPortList.add(nodeInfo); 39 | } 40 | 41 | return ipPortList; 42 | } 43 | 44 | } 45 | --------------------------------------------------------------------------------