├── .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 |
--------------------------------------------------------------------------------