diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java index e364b30d8..c1366dec2 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/CliService.java @@ -33,6 +33,18 @@ */ public interface CliService extends Lifecycle { + /** + * Reset the size of the read or write factor of a raft group in flexible mode + * + * @param groupId the raft group id + * @param conf current configuration + * @param readFactor read factor in flexible raft mode + * @param writeFactor write factor in flexible raft mode + * @return operation status + */ + Status resetFactor(final String groupId, final Configuration conf, final Integer readFactor, + final Integer writeFactor); + /** * Add a new peer into the replicating group which consists of |conf|. * return OK status when success. diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java index 6335ee1c6..13a9b7e5f 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/JRaftUtils.java @@ -25,6 +25,7 @@ import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.core.NodeImpl; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.BootstrapOptions; import com.alipay.sofa.jraft.util.Endpoint; @@ -112,6 +113,9 @@ public static Configuration getConfiguration(final String s) { return conf; } if (conf.parse(s)) { + conf.setEnableFlexible(false); + Quorum quorum = BallotFactory.buildMajorityQuorum(conf.size()); + conf.setQuorum(quorum); return conf; } throw new IllegalArgumentException("Invalid conf str:" + s); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java index fe89d9cd8..58f16b5e0 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/Node.java @@ -219,6 +219,18 @@ public interface Node extends Lifecycle, Describer { */ void changePeers(final Configuration newPeers, final Closure done); + /** + * This method can be called to reset the size of the read or write factor + * + * It should be noted that we cannot change the factory size while changing + * the number of cluster nodes. + * + * @param readFactor read factor for flexible raft + * @param writeFactor write factor for flexible raft + * @since 1.3.14 + */ + void resetFactor(final Integer readFactor, final Integer writeFactor, final Closure done); + /** * Reset the configuration of this node individually, without any replication * to other peers before this node becomes the leader. This function is diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/Quorum.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/Quorum.java new file mode 100644 index 000000000..b1fbc5b5a --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/Quorum.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft; + +/** + * @author Akai + */ +public class Quorum { + private int w; + + private int r; + + public Quorum(int w, int r) { + this.w = w; + this.r = r; + } + + public int getW() { + return w; + } + + public void setW(int w) { + this.w = w; + } + + public int getR() { + return r; + } + + public void setR(int r) { + this.r = r; + } + + @Override + public String toString() { + return "Quorum{w=" + w + ", r=" + r + '}'; + } + + @Override + public int hashCode() { + return super.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof Quorum) { + Quorum quorum = (Quorum) obj; + return quorum.getR() == this.getR() && quorum.getW() == this.getW(); + } + return false; + } + +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java index 71759806b..62095e82c 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/conf/Configuration.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; +import java.util.Objects; import java.util.Set; import org.apache.commons.lang.StringUtils; @@ -29,25 +30,35 @@ import org.slf4j.LoggerFactory; import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.util.Copiable; import com.alipay.sofa.jraft.util.Requires; /** * A configuration with a set of peers. * @author boyan (boyan@alibaba-inc.com) + * @author Akai * * 2018-Mar-15 11:00:26 AM */ public class Configuration implements Iterable, Copiable { - private static final Logger LOG = LoggerFactory.getLogger(Configuration.class); + private static final Logger LOG = LoggerFactory.getLogger(Configuration.class); - private static final String LEARNER_POSTFIX = "/learner"; + private static final String LEARNER_POSTFIX = "/learner"; - private List peers = new ArrayList<>(); + private Quorum quorum; + + private Integer readFactor; + + private Integer writeFactor; + + private Boolean isEnableFlexible = false; + + private List peers = new ArrayList<>(); // use LinkedHashSet to keep insertion order. - private LinkedHashSet learners = new LinkedHashSet<>(); + private LinkedHashSet learners = new LinkedHashSet<>(); public Configuration() { super(); @@ -68,16 +79,34 @@ public Configuration(final Iterable conf) { * @param conf configuration */ public Configuration(final Configuration conf) { - this(conf.getPeers(), conf.getLearners()); + this(conf.getPeers(), conf.getLearners(), conf.getQuorum(), conf.getReadFactor(), conf.getWriteFactor(), conf + .isEnableFlexible()); } /** * Construct a Configuration instance with peers and learners. * - * @param conf peers configuration - * @param learners learners - * @since 1.3.0 + * @param conf peers configuration + * @param learners learners + * @param quorum quorum + * @param readFactor read factor + * @param writeFactor write factor + * @param isEnableFlexible enable flexible mode or not + * @since 1.3.14 */ + public Configuration(final Iterable conf, final Iterable learners, final Quorum quorum, + final Integer readFactor, final Integer writeFactor, final Boolean isEnableFlexible) { + Requires.requireNonNull(conf, "conf"); + for (final PeerId peer : conf) { + this.peers.add(peer.copy()); + } + addLearners(learners); + this.quorum = quorum; + this.readFactor = readFactor; + this.writeFactor = writeFactor; + this.isEnableFlexible = isEnableFlexible; + } + public Configuration(final Iterable conf, final Iterable learners) { Requires.requireNonNull(conf, "conf"); for (final PeerId peer : conf) { @@ -86,6 +115,38 @@ public Configuration(final Iterable conf, final Iterable learner addLearners(learners); } + public Integer getReadFactor() { + return readFactor; + } + + public void setReadFactor(Integer readFactor) { + this.readFactor = readFactor; + } + + public Integer getWriteFactor() { + return writeFactor; + } + + public void setWriteFactor(Integer writeFactor) { + this.writeFactor = writeFactor; + } + + public Quorum getQuorum() { + return this.quorum; + } + + public void setQuorum(Quorum quorum) { + this.quorum = quorum; + } + + public Boolean isEnableFlexible() { + return isEnableFlexible; + } + + public void setEnableFlexible(Boolean enableFlexible) { + isEnableFlexible = enableFlexible; + } + public void setLearners(final LinkedHashSet learners) { this.learners = learners; } @@ -148,7 +209,7 @@ public List listLearners() { @Override public Configuration copy() { - return new Configuration(this.peers, this.learners); + return new Configuration(this); } /** @@ -251,12 +312,32 @@ public boolean equals(final Object obj) { if (this.peers == null) { return other.peers == null; } else { - return this.peers.equals(other.peers); + return this.peers.equals(other.peers) && Objects.equals(this.quorum, other.quorum) + && Objects.equals(this.readFactor, other.readFactor) + && Objects.equals(this.writeFactor, other.writeFactor) + && Objects.equals(this.isEnableFlexible, other.isEnableFlexible); } } @Override public String toString() { + StringBuilder sb = new StringBuilder(toBasicString()); + + if (Objects.nonNull(isEnableFlexible) && !isEmpty()) { + sb.append(",isEnableFlexible:").append(isEnableFlexible); + } + + if (Objects.nonNull(readFactor) || Objects.nonNull(writeFactor)) { + sb.append(",readFactor:").append(readFactor).append(",writeFactor:").append(writeFactor); + } + + if (Objects.nonNull(quorum)) { + sb.append(",quorum:").append(quorum); + } + return sb.toString(); + } + + public String toBasicString() { final StringBuilder sb = new StringBuilder(); final List peers = listPeers(); int i = 0; @@ -278,7 +359,6 @@ public String toString() { } i++; } - return sb.toString(); } @@ -311,9 +391,9 @@ public boolean parse(final String conf) { } /** - * Get the difference between |*this| and |rhs| - * |included| would be assigned to |*this| - |rhs| - * |excluded| would be assigned to |rhs| - |*this| + * Get the difference between |*this| and |rhs| + * |included| would be assigned to |*this| - |rhs| + * |excluded| would be assigned to |rhs| - |*this| */ public void diff(final Configuration rhs, final Configuration included, final Configuration excluded) { included.peers = new ArrayList<>(this.peers); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java index d24223beb..db48b63f7 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/BallotBox.java @@ -20,6 +20,8 @@ import javax.annotation.concurrent.ThreadSafe; +import com.alipay.sofa.jraft.entity.Ballot; +import com.alipay.sofa.jraft.entity.PeerId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,8 +30,6 @@ import com.alipay.sofa.jraft.Lifecycle; import com.alipay.sofa.jraft.closure.ClosureQueue; import com.alipay.sofa.jraft.conf.Configuration; -import com.alipay.sofa.jraft.entity.Ballot; -import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.BallotBoxOptions; import com.alipay.sofa.jraft.util.Describer; import com.alipay.sofa.jraft.util.OnlyForTest; @@ -114,12 +114,13 @@ public boolean commitAt(final long firstLogIndex, final long lastLogIndex, final final long startAt = Math.max(this.pendingIndex, firstLogIndex); Ballot.PosHint hint = new Ballot.PosHint(); for (long logIndex = startAt; logIndex <= lastLogIndex; logIndex++) { - final Ballot bl = this.pendingMetaQueue.get((int) (logIndex - this.pendingIndex)); - hint = bl.grant(peer, hint); - if (bl.isGranted()) { + final Ballot ballot = this.pendingMetaQueue.get((int) (logIndex - this.pendingIndex)); + hint = ballot.grant(peer, hint); + if (ballot.isGranted()) { lastCommittedIndex = logIndex; } } + if (lastCommittedIndex == 0) { return true; } @@ -198,8 +199,8 @@ public boolean resetPendingIndex(final long newPendingIndex) { * @return returns true on success */ public boolean appendPendingTask(final Configuration conf, final Configuration oldConf, final Closure done) { - final Ballot bl = new Ballot(); - if (!bl.init(conf, oldConf)) { + final Ballot ballot = new Ballot(); + if (!ballot.init(conf, oldConf)) { LOG.error("Fail to init ballot."); return false; } @@ -209,7 +210,7 @@ public boolean appendPendingTask(final Configuration conf, final Configuration o LOG.error("Node {} fail to appendingTask, pendingIndex={}.", this.opts.getNodeId(), this.pendingIndex); return false; } - this.pendingMetaQueue.add(bl); + this.pendingMetaQueue.add(ballot); this.closureQueue.appendPendingClosure(done); return true; } finally { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java index 3bc56b9ab..3b093e10d 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/CliServiceImpl.java @@ -28,6 +28,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import com.alipay.sofa.jraft.rpc.CliRequests; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,7 +113,8 @@ private void recordConfigurationChange(final String groupId, final List newPeer.parse(peerIdStr); newConf.addPeer(newPeer); } - LOG.info("Configuration of replication group {} changed from {} to {}.", groupId, oldConf, newConf); + LOG.info("Configuration of replication group {} changed from {} to {}.", groupId, oldConf.getPeers(), + newConf.getPeers()); } private Status checkLeaderAndConnect(final String groupId, final Configuration conf, final PeerId leaderId) { @@ -128,6 +130,38 @@ private Status checkLeaderAndConnect(final String groupId, final Configuration c return Status.OK(); } + @Override + public Status resetFactor(final String groupId, final Configuration conf, Integer readFactor, Integer writeFactor) { + final PeerId leaderId = new PeerId(); + final Status st = checkLeaderAndConnect(groupId, conf, leaderId); + if (!st.isOk()) { + return st; + } + final CliRequests.ResetFactorRequest.Builder rb = CliRequests.ResetFactorRequest.newBuilder() + .setGroupId(groupId) // + .setLeaderId(leaderId.toString()) // + .setReadFactor(readFactor) // + .setWriteFactor(writeFactor); + + try { + final Message result = this.cliClientService.resetFactor(leaderId.getEndpoint(), rb.build(), null).get(); + if (result instanceof CliRequests.ResetFactorResponse) { + final CliRequests.ResetFactorResponse resp = (CliRequests.ResetFactorResponse) result; + recordFactorChange(groupId, resp.getReadFactor(), resp.getWriteFactor()); + return Status.OK(); + } else { + return statusFromResponse(result); + } + + } catch (final Exception e) { + return new Status(-1, e.getMessage()); + } + } + + private void recordFactorChange(final String groupId, final Integer readFactor, final Integer writeFactor) { + LOG.info("Factor of group {} changed to readFactor:{} writeFactor:{}.", groupId, readFactor, writeFactor); + } + @Override public Status addPeer(final String groupId, final Configuration conf, final PeerId peer) { Requires.requireTrue(!StringUtils.isBlank(groupId), "Blank group id"); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java index e502458f1..ce1b9f49c 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/FSMCallerImpl.java @@ -18,10 +18,13 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -544,7 +547,12 @@ private void doCommitted(final long committedIndex) { if (logEntry.getType() == EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION) { if (logEntry.getOldPeers() != null && !logEntry.getOldPeers().isEmpty()) { // Joint stage is not supposed to be noticeable by end users. - this.fsm.onConfigurationCommitted(new Configuration(iterImpl.entry().getPeers())); + Configuration conf = new Configuration(iterImpl.entry().getPeers()); + conf.setEnableFlexible(logEntry.getEnableFlexible()); + conf.setReadFactor(logEntry.getReadFactor()); + conf.setWriteFactor(logEntry.getWriteFactor()); + conf.setQuorum(convertToQuorum(logEntry)); + this.fsm.onConfigurationCommitted(conf); } } if (iterImpl.done() != null) { @@ -612,6 +620,7 @@ private void doSnapshotSave(final SaveSnapshotClosure done) { final RaftOutter.SnapshotMeta.Builder metaBuilder = RaftOutter.SnapshotMeta.newBuilder() // .setLastIncludedIndex(lastAppliedIndex) // .setLastIncludedTerm(this.lastAppliedTerm); + final LogOutter.Quorum.Builder quorumBuilder = LogOutter.Quorum.newBuilder(); final ConfigurationEntry confEntry = this.logManager.getConfiguration(lastAppliedIndex); if (confEntry == null || confEntry.isEmpty()) { LOG.error("Empty conf entry for lastAppliedIndex={}", lastAppliedIndex); @@ -625,6 +634,20 @@ private void doSnapshotSave(final SaveSnapshotClosure done) { for (final PeerId peer : confEntry.getConf().getLearners()) { metaBuilder.addLearners(peer.toString()); } + + Configuration conf = confEntry.getConf(); + metaBuilder.setIsEnableFlexible(conf.isEnableFlexible()); + // set new factor + if (Objects.nonNull(conf.getReadFactor())) { + metaBuilder.setReadFactor(conf.getReadFactor()); + } + if (Objects.nonNull(conf.getWriteFactor())) { + metaBuilder.setWriteFactor(conf.getWriteFactor()); + } + if (Objects.nonNull(conf.getQuorum())) { + LogOutter.Quorum quorum = quorumBuilder.setR(conf.getQuorum().getR()).setW(conf.getQuorum().getW()).build(); + metaBuilder.setQuorum(quorum); + } if (confEntry.getOldConf() != null) { for (final PeerId peer : confEntry.getOldConf()) { metaBuilder.addOldPeers(peer.toString()); @@ -632,6 +655,19 @@ private void doSnapshotSave(final SaveSnapshotClosure done) { for (final PeerId peer : confEntry.getOldConf().getLearners()) { metaBuilder.addOldLearners(peer.toString()); } + Configuration oldConf = confEntry.getOldConf(); + // set old Quorum + if (Objects.nonNull(oldConf.getReadFactor())) { + metaBuilder.setOldReadFactor(oldConf.getReadFactor()); + } + if (Objects.nonNull(oldConf.getWriteFactor())) { + metaBuilder.setOldWriteFactor(oldConf.getWriteFactor()); + } + if (!oldConf.isEmpty()) { + LogOutter.Quorum oldQuorum = quorumBuilder.setR(oldConf.getQuorum().getR()) + .setW(oldConf.getQuorum().getW()).build(); + metaBuilder.setOldQuorum(oldQuorum); + } } final SnapshotWriter writer = done.start(metaBuilder.build()); if (writer == null) { @@ -722,6 +758,12 @@ private void doSnapshotLoad(final LoadSnapshotClosure done) { Requires.requireTrue(peer.parse(meta.getPeers(i)), "Parse peer failed"); conf.addPeer(peer); } + // set Quorum from meta + conf.setWriteFactor(meta.getWriteFactor()); + conf.setReadFactor(meta.getReadFactor()); + conf.setEnableFlexible(meta.getIsEnableFlexible()); + conf.setQuorum(convertToQuorum(meta)); + this.fsm.onConfigurationCommitted(conf); } this.lastCommittedIndex.set(meta.getLastIncludedIndex()); @@ -730,6 +772,13 @@ private void doSnapshotLoad(final LoadSnapshotClosure done) { done.run(Status.OK()); } + private Quorum convertToQuorum(RaftOutter.SnapshotMeta meta){ + return new Quorum(meta.getQuorum().getW(), meta.getQuorum().getR()); + } + private Quorum convertToQuorum(LogEntry entry){ + return new Quorum(entry.getQuorum().getW(), entry.getQuorum().getR()); + } + private void doOnError(final OnErrorClosure done) { setError(done.getError()); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java index ae1c0df4a..0d9f4bbdd 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/NodeImpl.java @@ -44,6 +44,7 @@ import com.alipay.sofa.jraft.JRaftUtils; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.NodeManager; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.ReadOnlyService; import com.alipay.sofa.jraft.ReplicatorGroup; import com.alipay.sofa.jraft.Status; @@ -56,6 +57,7 @@ import com.alipay.sofa.jraft.conf.ConfigurationEntry; import com.alipay.sofa.jraft.conf.ConfigurationManager; import com.alipay.sofa.jraft.entity.Ballot; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.EnumOutter; import com.alipay.sofa.jraft.entity.LeaderChangeContext; import com.alipay.sofa.jraft.entity.LogEntry; @@ -65,6 +67,7 @@ import com.alipay.sofa.jraft.entity.RaftOutter; import com.alipay.sofa.jraft.entity.Task; import com.alipay.sofa.jraft.entity.UserLog; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import com.alipay.sofa.jraft.error.LogIndexOutOfBoundsException; import com.alipay.sofa.jraft.error.LogNotFoundException; import com.alipay.sofa.jraft.error.OverloadException; @@ -179,6 +182,7 @@ public class NodeImpl implements Node, RaftServerService { private PeerId votedId; private final Ballot voteCtx = new Ballot(); private final Ballot prevVoteCtx = new Ballot(); + private ConfigurationEntry conf; private StopTransferArg stopTransferArg; /** Raft group and node options and identifier */ @@ -333,7 +337,8 @@ enum Stage { STAGE_NONE, // none stage STAGE_CATCHING_UP, // the node is catching-up STAGE_JOINT, // joint stage - STAGE_STABLE // stable stage + STAGE_STABLE, // stable stage + STAGE_RESET_FACTOR // reset factor stage } final NodeImpl node; @@ -348,6 +353,13 @@ enum Stage { // learners List newLearners = new ArrayList<>(); List oldLearners = new ArrayList<>(); + Integer readFactor; + Integer writeFactor; + Integer oldReadFactor; + Integer oldWriteFactor; + Boolean isEnableFlexible; + Quorum quorum; + Quorum oldQuorum; Closure done; public ConfigurationCtx(final NodeImpl node) { @@ -358,6 +370,14 @@ public ConfigurationCtx(final NodeImpl node) { this.done = null; } + Configuration buildConfiguration() { + return new Configuration(newPeers, newLearners, quorum, readFactor, writeFactor, isEnableFlexible); + } + + Configuration buildOldConfiguration() { + return new Configuration(oldPeers, oldLearners, oldQuorum, oldReadFactor, oldWriteFactor, isEnableFlexible); + } + /** * Start change configuration. */ @@ -382,11 +402,24 @@ void start(final Configuration oldConf, final Configuration newConf, final Closu this.newPeers = newConf.listPeers(); this.oldLearners = oldConf.listLearners(); this.newLearners = newConf.listLearners(); + this.readFactor = newConf.getReadFactor(); + this.writeFactor = newConf.getWriteFactor(); + this.quorum = newConf.getQuorum(); + this.oldReadFactor = oldConf.getReadFactor(); + this.oldWriteFactor = oldConf.getWriteFactor(); + this.oldQuorum = oldConf.getQuorum(); + this.isEnableFlexible = newConf.isEnableFlexible(); final Configuration adding = new Configuration(); final Configuration removing = new Configuration(); newConf.diff(oldConf, adding, removing); this.nchanges = adding.size() + removing.size(); - + // Determine as resetFactor API + if (newConf.isEnableFlexible() && nchanges == 0 + && ((!Objects.equals(readFactor, oldReadFactor)) || !Objects.equals(writeFactor, oldWriteFactor))) { + this.stage = Stage.STAGE_RESET_FACTOR; + nextStage(); + return; + } addNewLearners(); if (adding.isEmpty()) { nextStage(); @@ -490,6 +523,10 @@ void flush(final Configuration conf, final Configuration oldConf) { Requires.requireTrue(!isBusy(), "Flush when busy"); this.newPeers = conf.listPeers(); this.newLearners = conf.listLearners(); + this.readFactor = conf.getReadFactor(); + this.writeFactor = conf.getWriteFactor(); + this.quorum = conf.getQuorum(); + this.isEnableFlexible = conf.isEnableFlexible(); if (oldConf == null || oldConf.isEmpty()) { this.stage = Stage.STAGE_STABLE; this.oldPeers = this.newPeers; @@ -498,6 +535,9 @@ void flush(final Configuration conf, final Configuration oldConf) { this.stage = Stage.STAGE_JOINT; this.oldPeers = oldConf.listPeers(); this.oldLearners = oldConf.listLearners(); + this.oldReadFactor = oldConf.getReadFactor(); + this.oldWriteFactor = oldConf.getWriteFactor(); + this.oldQuorum = oldConf.getQuorum(); } this.node.unsafeApplyConfiguration(conf, oldConf == null || oldConf.isEmpty() ? null : oldConf, true); } @@ -508,13 +548,12 @@ void nextStage() { case STAGE_CATCHING_UP: if (this.nchanges > 0) { this.stage = Stage.STAGE_JOINT; - this.node.unsafeApplyConfiguration(new Configuration(this.newPeers, this.newLearners), - new Configuration(this.oldPeers), false); + this.node.unsafeApplyConfiguration(buildConfiguration(), buildOldConfiguration(), false); return; } case STAGE_JOINT: this.stage = Stage.STAGE_STABLE; - this.node.unsafeApplyConfiguration(new Configuration(this.newPeers, this.newLearners), null, false); + this.node.unsafeApplyConfiguration(buildConfiguration(), null, false); break; case STAGE_STABLE: final boolean shouldStepDown = !this.newPeers.contains(this.node.serverId); @@ -524,6 +563,10 @@ void nextStage() { "This node was removed.")); } break; + case STAGE_RESET_FACTOR: + this.stage = Stage.STAGE_STABLE; + this.node.unsafeApplyConfiguration(buildConfiguration(), buildOldConfiguration(), false); + break; case STAGE_NONE: // noinspection ConstantConditions Requires.requireTrue(false, "Can't reach here"); @@ -726,6 +769,8 @@ private void checkAndSetConfiguration(final boolean inLock) { prevTargetPriority, this.targetPriority); } this.electionTimeoutCounter = 0; + // Refresh voteCtx And preVoteCtx + refreshVoteCtx(this.conf.getConf(), this.conf.getOldConf()); } } finally { if (!inLock) { @@ -734,6 +779,19 @@ private void checkAndSetConfiguration(final boolean inLock) { } } + private void refreshVoteCtx(Configuration conf, Configuration oldConf) { + this.prevVoteCtx.refreshBallot(conf, oldConf); + this.voteCtx.refreshBallot(conf, oldConf); + LOG.info("Refresh Ballot newConf {} and oldConf {}", conf, oldConf); + } + + private Configuration rebuildConfiguration(List peers, Configuration conf) { + Quorum newQuorum = conf.isEnableFlexible() ? BallotFactory.buildFlexibleQuorum(conf.getReadFactor(), + conf.getWriteFactor(), peers.size()) : BallotFactory.buildMajorityQuorum(peers.size()); + return new Configuration(peers, conf.getLearners(), newQuorum, conf.getReadFactor(), conf.getWriteFactor(), + conf.isEnableFlexible()); + } + /** * Get max priority value for all nodes in the same Raft group, and update current node's target priority value. * @@ -839,6 +897,11 @@ public boolean bootstrap(final BootstrapOptions opts) throws InterruptedExceptio entry.getId().setTerm(this.currTerm); entry.setPeers(opts.getGroupConf().listPeers()); entry.setLearners(opts.getGroupConf().listLearners()); + Quorum quorum = BallotFactory.buildMajorityQuorum(opts.getGroupConf().listPeers().size()); + final LogOutter.Quorum.Builder quorumBuilder = LogOutter.Quorum.newBuilder(); + LogOutter.Quorum logOutterQuorum = quorumBuilder.setR(quorum.getR()).setW(quorum.getW()).build(); + entry.setQuorum(logOutterQuorum); + entry.setEnableFlexible(false); final List entries = new ArrayList<>(); entries.add(entry); @@ -887,6 +950,10 @@ private int randomTimeout(final int timeoutMs) { return ThreadLocalRandom.current().nextInt(timeoutMs, timeoutMs + this.raftOptions.getMaxElectionDelayMs()); } + private boolean checkFactor(Integer writeFactor, Integer readFactor) { + return BallotFactory.checkValid(readFactor, writeFactor); + } + @Override public boolean init(final NodeOptions opts) { Requires.requireNonNull(opts, "Null node options"); @@ -898,7 +965,13 @@ public boolean init(final NodeOptions opts) { this.metrics = new NodeMetrics(opts.isEnableMetrics()); this.serverId.setPriority(opts.getElectionPriority()); this.electionTimeoutCounter = 0; + Configuration initialConf = options.getInitialConf(); + + if (initialConf.isEnableFlexible() + && !checkFactor(initialConf.getWriteFactor(), initialConf.getReadFactor())) { + return false; + } if (this.serverId.getIp().equals(Utils.IP_ANY)) { LOG.error("Node can't started from IP_ANY."); return false; @@ -1013,6 +1086,40 @@ protected int adjustTimeout(final int timeoutMs) { LOG.error("Node {} initFSMCaller failed.", getNodeId()); return false; } + + this.conf = new ConfigurationEntry(); + this.conf.setId(new LogId()); + // if have log using conf in log, else using conf in options + if (this.logManager.getLastLogIndex() > 0) { + checkAndSetConfiguration(false); + } else { + this.conf.setConf(this.options.getInitialConf()); + // initially set to max(priority of all nodes) + this.targetPriority = getMaxPriorityOfNodes(this.conf.getConf().getPeers()); + } + + if (!this.conf.isEmpty()) { + Requires.requireTrue(this.conf.isValid(), "Invalid conf: %s", this.conf); + } else { + LOG.info("Init node {} with empty conf.", this.serverId); + } + // Initialize majority mode + if (Objects.isNull(conf.getConf().getQuorum()) && !conf.getConf().isEnableFlexible()) { + Quorum quorum = BallotFactory.buildMajorityQuorum(conf.getConf().size()); + conf.getConf().setQuorum(quorum); + } + + // init prevVoteCtx + if (!prevVoteCtx.init(conf.getConf(), conf.getOldConf())) { + LOG.error("Fail to init prevVoteCtx."); + return false; + } + // init voteCtx + if (!voteCtx.init(conf.getConf(), conf.getOldConf())) { + LOG.error("Fail to init voteCtx."); + return false; + } + this.ballotBox = new BallotBox(); final BallotBoxOptions ballotBoxOpts = new BallotBoxOptions(); ballotBoxOpts.setWaiter(this.fsmCaller); @@ -1033,22 +1140,6 @@ protected int adjustTimeout(final int timeoutMs) { LOG.error("Node {} is initialized with inconsistent log, status={}.", getNodeId(), st); return false; } - this.conf = new ConfigurationEntry(); - this.conf.setId(new LogId()); - // if have log using conf in log, else using conf in options - if (this.logManager.getLastLogIndex() > 0) { - checkAndSetConfiguration(false); - } else { - this.conf.setConf(this.options.getInitialConf()); - // initially set to max(priority of all nodes) - this.targetPriority = getMaxPriorityOfNodes(this.conf.getConf().getPeers()); - } - - if (!this.conf.isEmpty()) { - Requires.requireTrue(this.conf.isValid(), "Invalid conf: %s", this.conf); - } else { - LOG.info("Init node {} with empty conf.", this.serverId); - } // TODO RPC service and ReplicatorGroup is in cycle dependent, refactor it this.replicatorGroup = new ReplicatorGroupImpl(); @@ -1147,7 +1238,6 @@ private void electSelf() { this.votedId = this.serverId.copy(); LOG.debug("Node {} start vote timer, term={} .", getNodeId(), this.currTerm); this.voteTimer.start(); - this.voteCtx.init(this.conf.getConf(), this.conf.isStable() ? null : this.conf.getOldConf()); oldTerm = this.currTerm; } finally { this.writeLock.unlock(); @@ -1385,17 +1475,18 @@ private void executeApplyingTasks(final List tasks) { final LogEntryAndClosure task = tasks.get(i); if (task.expectedTerm != -1 && task.expectedTerm != this.currTerm) { LOG.debug("Node {} can't apply task whose expectedTerm={} doesn't match currTerm={}.", getNodeId(), - task.expectedTerm, this.currTerm); + task.expectedTerm, this.currTerm); if (task.done != null) { final Status st = new Status(RaftError.EPERM, "expected_term=%d doesn't match current_term=%d", - task.expectedTerm, this.currTerm); + task.expectedTerm, this.currTerm); ThreadPoolsFactory.runClosureInThread(this.groupId, task.done, st); task.reset(); } continue; } + if (!this.ballotBox.appendPendingTask(this.conf.getConf(), - this.conf.isStable() ? null : this.conf.getOldConf(), task.done)) { + this.conf.isStable() ? null : this.conf.getOldConf(), task.done)) { ThreadPoolsFactory.runClosureInThread(this.groupId, task.done, new Status(RaftError.EINTERNAL, "Fail to append task.")); task.reset(); continue; @@ -1426,8 +1517,9 @@ public NodeMetrics getNodeMetrics() { /** * Returns the JRaft service factory for current node. - * @since 1.2.6 + * * @return the service factory + * @since 1.2.6 */ public JRaftServiceFactory getServiceFactory() { return this.serviceFactory; @@ -1456,20 +1548,20 @@ public void readIndex(ReadOnlyOption readOnlyOptions, byte[] requestContext, Rea private class ReadIndexHeartbeatResponseClosure extends RpcResponseClosureAdapter { final ReadIndexResponse.Builder respBuilder; final RpcResponseClosure closure; - final int quorum; + final Quorum quorum; final int failPeersThreshold; int ackSuccess; int ackFailures; boolean isDone; public ReadIndexHeartbeatResponseClosure(final RpcResponseClosure closure, - final ReadIndexResponse.Builder rb, final int quorum, + final ReadIndexResponse.Builder rb, final Quorum quorum, final int peersCount) { super(); this.closure = closure; this.respBuilder = rb; this.quorum = quorum; - this.failPeersThreshold = peersCount % 2 == 0 ? (quorum - 1) : quorum; + this.failPeersThreshold = peersCount - quorum.getR() + 1; this.ackSuccess = 0; this.ackFailures = 0; this.isDone = false; @@ -1486,12 +1578,14 @@ public synchronized void run(final Status status) { this.ackFailures++; } // Include leader self vote yes. - if (this.ackSuccess + 1 >= this.quorum) { + if (this.ackSuccess + 1 >= this.quorum.getR()) { + LOG.info("Reading successfully..."); this.respBuilder.setSuccess(true); this.closure.setResponse(this.respBuilder.build()); this.closure.run(Status.OK()); this.isDone = true; } else if (this.ackFailures >= this.failPeersThreshold) { + LOG.info("Reading failed..."); this.respBuilder.setSuccess(false); this.closure.setResponse(this.respBuilder.build()); this.closure.run(Status.OK()); @@ -1529,12 +1623,12 @@ public void handleReadIndexRequest(final ReadIndexRequest request, final RpcResp } } - private int getQuorum() { + private int getReadQuorum() { final Configuration c = this.conf.getConf(); if (c.isEmpty()) { return 0; } - return c.getPeers().size() / 2 + 1; + return this.conf.getConf().getQuorum().getR(); } private void readFollower(final ReadIndexRequest request, final RpcResponseClosure closure) { @@ -1552,8 +1646,7 @@ private void readFollower(final ReadIndexRequest request, final RpcResponseClosu private void readLeader(final ReadIndexRequest request, final ReadIndexResponse.Builder respBuilder, final RpcResponseClosure closure) { - final int quorum = getQuorum(); - if (quorum <= 1) { + if (this.conf.getConf().getQuorum().getR() <= 1) { // Only one peer, fast path. respBuilder.setSuccess(true) // .setIndex(this.ballotBox.getLastCommittedIndex()); @@ -1574,7 +1667,7 @@ private void readLeader(final ReadIndexRequest request, final ReadIndexResponse. } respBuilder.setIndex(lastCommittedIndex); - if (request.getPeerId() != null) { + if (Objects.nonNull(request.getPeerId())) { // request from follower or learner, check if the follower/learner is in current conf. final PeerId peer = new PeerId(); peer.parse(request.getServerId()); @@ -1597,7 +1690,7 @@ private void readLeader(final ReadIndexRequest request, final ReadIndexResponse. final List peers = this.conf.getConf().getPeers(); Requires.requireTrue(peers != null && !peers.isEmpty(), "Empty peers"); final ReadIndexHeartbeatResponseClosure heartbeatDone = new ReadIndexHeartbeatResponseClosure(closure, - respBuilder, quorum, peers.size()); + respBuilder, this.conf.getConf().getQuorum(), peers.size()); // Send heartbeat requests to followers for (final PeerId peer : peers) { if (peer.equals(this.serverId)) { @@ -1627,29 +1720,29 @@ public void apply(final Task task) { entry.setData(task.getData()); final EventTranslator translator = (event, sequence) -> { - event.reset(); - event.done = task.getDone(); - event.entry = entry; - event.expectedTerm = task.getExpectedTerm(); + event.reset(); + event.done = task.getDone(); + event.entry = entry; + event.expectedTerm = task.getExpectedTerm(); }; - switch(this.options.getApplyTaskMode()) { - case Blocking: - this.applyQueue.publishEvent(translator); - break; - case NonBlocking: - default: - if (!this.applyQueue.tryPublishEvent(translator)) { - String errorMsg = "Node is busy, has too many tasks, queue is full and bufferSize="+ this.applyQueue.getBufferSize(); - ThreadPoolsFactory.runClosureInThread(this.groupId, task.getDone(), - new Status(RaftError.EBUSY, errorMsg)); - LOG.warn("Node {} applyQueue is overload.", getNodeId()); - this.metrics.recordTimes("apply-task-overload-times", 1); - if(task.getDone() == null) { - throw new OverloadException(errorMsg); - } - } - break; + switch (this.options.getApplyTaskMode()) { + case Blocking: + this.applyQueue.publishEvent(translator); + break; + case NonBlocking: + default: + if (!this.applyQueue.tryPublishEvent(translator)) { + String errorMsg = "Node is busy, has too many tasks, queue is full and bufferSize=" + this.applyQueue.getBufferSize(); + ThreadPoolsFactory.runClosureInThread(this.groupId, task.getDone(), + new Status(RaftError.EBUSY, errorMsg)); + LOG.warn("Node {} applyQueue is overload.", getNodeId()); + this.metrics.recordTimes("apply-task-overload-times", 1); + if (task.getDone() == null) { + throw new OverloadException(errorMsg); + } + } + break; } } @@ -2129,6 +2222,14 @@ private void fillLogEntryPeers(final RaftOutter.EntryMeta entry, final LogEntry } logEntry.setOldLearners(peers); } + + logEntry.setEnableFlexible(this.conf.getConf().isEnableFlexible()); + logEntry.setReadFactor(entry.getReadFactor()); + logEntry.setWriteFactor(entry.getWriteFactor()); + logEntry.setOldReadFactor(entry.getOldReadFactor()); + logEntry.setOldWriteFactor(entry.getOldWriteFactor()); + logEntry.setQuorum(entry.getQuorum()); + logEntry.setOldQuorum(entry.getOldQuorum()); } // called when leader receive greater term in AppendEntriesResponse @@ -2220,8 +2321,11 @@ private boolean checkDeadNodes(final Configuration conf, final long monotonicNow LOG.warn("Node {} steps down when alive nodes don't satisfy quorum, term={}, deadNodes={}, conf={}.", getNodeId(), this.currTerm, deadNodes, conf); final Status status = new Status(); - status.setError(RaftError.ERAFTTIMEDOUT, "Majority of the group dies: %d/%d", deadNodes.size(), - peers.size()); + String msg = this.conf.getConf().isEnableFlexible() ? "Reading quorum does not meet availability conditions: " + + getReadQuorum() + + ", Some nodes in the cluster dies" + : "Majority of the group dies"; + status.setError(RaftError.ERAFTTIMEDOUT, "%s: %d/%d", msg, deadNodes.size(), peers.size()); stepDown(this.currTerm, false, status); } return false; @@ -2252,7 +2356,14 @@ private boolean checkDeadNodes0(final List peers, final long monotonicNo deadNodes.addPeer(peer); } } - if (aliveCount >= peers.size() / 2 + 1) { + + // If the writeFactor in a cluster is less than readFactor and the number of nodes + // is less than r and greater than or equal to w, we hope to still be in a writable state. + // Therefore, read requests may fail at this time, but the cluster is still available + Quorum quorum = this.conf.getConf().getQuorum(); + int targetCount = this.conf.getConf().isEnableFlexible() && quorum.getW() < quorum.getR() ? quorum.getW() + : quorum.getR(); + if (aliveCount >= targetCount) { updateLastLeaderTimestamp(startLease); return true; } @@ -2350,12 +2461,28 @@ private void unsafeApplyConfiguration(final Configuration newConf, final Configu final boolean leaderStart) { Requires.requireTrue(this.confCtx.isBusy(), "ConfigurationContext is not busy"); final LogEntry entry = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); + + final LogOutter.Quorum.Builder quorumBuilder = LogOutter.Quorum.newBuilder(); + LogOutter.Quorum quorum = quorumBuilder.setR(newConf.getQuorum().getR()).setW(newConf.getQuorum().getW()) + .build(); entry.setId(new LogId(0, this.currTerm)); entry.setPeers(newConf.listPeers()); entry.setLearners(newConf.listLearners()); + entry.setReadFactor(newConf.getReadFactor()); + entry.setWriteFactor(newConf.getWriteFactor()); + entry.setEnableFlexible(newConf.isEnableFlexible()); + entry.setQuorum(quorum); + if (oldConf != null) { entry.setOldPeers(oldConf.listPeers()); entry.setOldLearners(oldConf.listLearners()); + entry.setOldReadFactor(oldConf.getReadFactor()); + entry.setOldWriteFactor(oldConf.getWriteFactor()); + if (!oldConf.isEmpty()) { + LogOutter.Quorum oldQuorum = quorumBuilder.setR(oldConf.getQuorum().getR()) + .setW(oldConf.getQuorum().getW()).build(); + entry.setOldQuorum(oldQuorum); + } } final ConfigurationChangeDone configurationChangeDone = new ConfigurationChangeDone(this.currTerm, leaderStart); // Use the new_conf to deal the quorum of this very log @@ -2371,7 +2498,6 @@ private void unsafeApplyConfiguration(final Configuration newConf, final Configu } private void unsafeRegisterConfChange(final Configuration oldConf, final Configuration newConf, final Closure done) { - Requires.requireTrue(newConf.isValid(), "Invalid new conf: %s", newConf); // The new conf entry(will be stored in log manager) should be valid Requires.requireTrue(new ConfigurationEntry(null, newConf, oldConf).isValid(), "Invalid conf entry: %s", @@ -2399,7 +2525,7 @@ private void unsafeRegisterConfChange(final Configuration oldConf, final Configu } return; } - // Return immediately when the new peers equals to current configuration + // Return immediately when the new peers equal to current configuration if (this.conf.getConf().equals(newConf)) { ThreadPoolsFactory.runClosureInThread(this.groupId, done, Status.OK()); return; @@ -2806,7 +2932,7 @@ public void shutdown(Closure done) { final CountDownLatch latch = new CountDownLatch(1); this.shutdownLatch = latch; ThreadPoolsFactory.runInThread(this.groupId, - () -> this.applyQueue.publishEvent((event, sequence) -> event.shutdownLatch = latch)); + () -> this.applyQueue.publishEvent((event, sequence) -> event.shutdownLatch = latch)); } else { final int num = GLOBAL_NUM_NODES.decrementAndGet(); LOG.info("The number of active nodes decrement to {}.", num); @@ -2833,18 +2959,18 @@ public void shutdown(Closure done) { // Call join() asynchronously final Closure shutdownHook = done; ThreadPoolsFactory.runInThread(this.groupId, () -> { - try { - join(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } finally { - // This node is down, it's ok to invoke done right now. Don't invoke this - // in place to avoid the dead writeLock issue when done.Run() is going to acquire - // a writeLock which is already held by the caller - if (shutdownHook != null) { - shutdownHook.run(Status.OK()); + try { + join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + // This node is down, it's ok to invoke done right now. Don't invoke this + // in place to avoid the dead writeLock issue when done.Run() is going to acquire + // a writeLock which is already held by the caller + if (shutdownHook != null) { + shutdownHook.run(Status.OK()); + } } - } }); } @@ -3016,9 +3142,9 @@ public void addPeer(final PeerId peer, final Closure done) { this.writeLock.lock(); try { Requires.requireTrue(!this.conf.getConf().contains(peer), "Peer already exists in current configuration"); - - final Configuration newConf = new Configuration(this.conf.getConf()); - newConf.addPeer(peer); + List peers = new ArrayList<>(this.conf.getConf().getPeers()); + peers.add(peer); + final Configuration newConf = rebuildConfiguration(peers, this.conf.getConf()); unsafeRegisterConfChange(this.conf.getConf(), newConf, done); } finally { this.writeLock.unlock(); @@ -3031,9 +3157,9 @@ public void removePeer(final PeerId peer, final Closure done) { this.writeLock.lock(); try { Requires.requireTrue(this.conf.getConf().contains(peer), "Peer not found in current configuration"); - - final Configuration newConf = new Configuration(this.conf.getConf()); - newConf.removePeer(peer); + List peers = new ArrayList<>(this.conf.getConf().getPeers()); + peers.remove(peer); + final Configuration newConf = rebuildConfiguration(peers, this.conf.getConf()); unsafeRegisterConfChange(this.conf.getConf(), newConf, done); } finally { this.writeLock.unlock(); @@ -3046,8 +3172,31 @@ public void changePeers(final Configuration newPeers, final Closure done) { Requires.requireTrue(!newPeers.isEmpty(), "Empty new peers"); this.writeLock.lock(); try { - LOG.info("Node {} change peers from {} to {}.", getNodeId(), this.conf.getConf(), newPeers); - unsafeRegisterConfChange(this.conf.getConf(), newPeers, done); + List peers = newPeers.getPeers(); + final Configuration newConf = rebuildConfiguration(peers, this.conf.getConf()); + LOG.info("Node {} change peers from {} to {}.", getNodeId(), this.conf.getConf(), newConf); + unsafeRegisterConfChange(this.conf.getConf(), newConf, done); + } finally { + this.writeLock.unlock(); + } + } + + @Override + public void resetFactor(Integer readFactor, Integer writeFactor, Closure done) { + Requires.requireTrue(this.conf.getConf().isEnableFlexible(), + "Current raft cluster has not enabled flexible mode"); + Requires.requireTrue(checkFactor(writeFactor, readFactor), "Factor check fail"); + try { + this.writeLock.lock(); + Configuration oldConf = this.conf.getConf(); + Configuration newConf = new Configuration(oldConf); + Quorum quorum = BallotFactory.buildFlexibleQuorum(readFactor, writeFactor, oldConf.getPeers().size()); + newConf.setReadFactor(readFactor); + newConf.setWriteFactor(writeFactor); + newConf.setQuorum(quorum); + newConf.setEnableFlexible(true); + LOG.info("reset new factor to followers newConf:{} oldConf:{}", newConf, oldConf); + unsafeRegisterConfChange(oldConf, newConf, done); } finally { this.writeLock.unlock(); } @@ -3088,6 +3237,8 @@ public Status resetPeers(final Configuration newPeers) { this.conf.setConf(newConf); this.conf.getOldConf().reset(); stepDown(this.currTerm + 1, false, new Status(RaftError.ESETPEER, "Raft node set peer normally")); + // Refresh voteCtx And preVoteCtx + refreshVoteCtx(this.conf.getConf(), this.conf.getOldConf()); return Status.OK(); } finally { this.writeLock.unlock(); @@ -3194,7 +3345,7 @@ public Status transferLeadershipTo(final PeerId peer) { try { if (this.state != State.STATE_LEADER) { LOG.warn("Node {} can't transfer leadership to peer {} as it is in state {}.", getNodeId(), peer, - this.state); + this.state); return new Status(this.state == State.STATE_TRANSFERRING ? RaftError.EBUSY : RaftError.EPERM, "Not a leader"); } @@ -3212,8 +3363,8 @@ public Status transferLeadershipTo(final PeerId peer) { // completed so that the peer's configuration is up-to-date when it // receives the TimeOutNowRequest. LOG.warn( - "Node {} refused to transfer leadership to peer {} when the leader is changing the configuration.", - getNodeId(), peer); + "Node {} refused to transfer leadership to peer {} when the leader is changing the configuration.", + getNodeId(), peer); return new Status(RaftError.EBUSY, "Changing the configuration"); } @@ -3232,7 +3383,7 @@ public Status transferLeadershipTo(final PeerId peer) { } if (!this.conf.contains(peerId)) { LOG.info("Node {} refused to transfer leadership to peer {} as it is not in {}.", getNodeId(), peer, - this.conf); + this.conf); return new Status(RaftError.EINVAL, "Not in current configuration"); } @@ -3243,13 +3394,13 @@ public Status transferLeadershipTo(final PeerId peer) { } this.state = State.STATE_TRANSFERRING; final Status status = new Status(RaftError.ETRANSFERLEADERSHIP, - "Raft leader is transferring leadership to %s", peerId); + "Raft leader is transferring leadership to %s", peerId); onLeaderStop(status); LOG.info("Node {} starts to transfer leadership to peer {}.", getNodeId(), peer); final StopTransferArg stopArg = new StopTransferArg(this, this.currTerm, peerId); this.stopTransferArg = stopArg; this.transferTimer = this.timerManager.schedule(() -> onTransferTimeout(stopArg), - this.options.getElectionTimeoutMs(), TimeUnit.MILLISECONDS); + this.options.getElectionTimeoutMs(), TimeUnit.MILLISECONDS); } finally { this.writeLock.unlock(); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/Replicator.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/Replicator.java index f44543826..bd1b40630 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/core/Replicator.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/core/Replicator.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.PriorityQueue; import java.util.concurrent.Future; import java.util.concurrent.ScheduledFuture; @@ -870,10 +871,22 @@ private void fillMetaPeers(final RaftOutter.EntryMeta.Builder emb, final LogEntr for (final PeerId peer : entry.getPeers()) { emb.addPeers(peer.toString()); } + if(entry.haveFactorValue()){ + emb.setReadFactor(entry.getReadFactor()); + emb.setWriteFactor(entry.getWriteFactor()); + } + emb.setQuorum(entry.getQuorum()); if (entry.getOldPeers() != null) { for (final PeerId peer : entry.getOldPeers()) { emb.addOldPeers(peer.toString()); } + if(Objects.nonNull(entry.getOldReadFactor())) { + emb.setOldReadFactor(entry.getOldReadFactor()); + } + if(Objects.nonNull(entry.getOldWriteFactor())) { + emb.setOldWriteFactor(entry.getOldWriteFactor()); + } + emb.setOldQuorum(entry.getOldQuorum()); } if (entry.getLearners() != null) { for (final PeerId peer : entry.getLearners()) { @@ -885,6 +898,9 @@ private void fillMetaPeers(final RaftOutter.EntryMeta.Builder emb, final LogEntr emb.addOldLearners(peer.toString()); } } + if(Objects.nonNull(entry.getEnableFlexible())){ + emb.setIsEnableFlexible(entry.getEnableFlexible()); + } } public static ThreadId start(final ReplicatorOptions opts, final RaftOptions raftOptions) { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/Ballot.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/Ballot.java index 5f9f1eb0b..370ec7515 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/Ballot.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/Ballot.java @@ -16,10 +16,14 @@ */ package com.alipay.sofa.jraft.entity; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.ArrayList; import java.util.List; - -import com.alipay.sofa.jraft.conf.Configuration; +import java.util.Objects; /** * A ballot to vote. @@ -29,6 +33,7 @@ * 2018-Mar-15 2:29:11 PM */ public class Ballot { + private static final Logger LOG = LoggerFactory.getLogger(Ballot.class); public static final class PosHint { int pos0 = -1; // position in current peers @@ -53,6 +58,9 @@ public UnfoundPeerId(PeerId peerId, int index, boolean found) { private final List oldPeers = new ArrayList<>(); private int oldQuorum; + public Ballot() { + } + /** * Init the ballot with current conf and old conf. * @@ -65,13 +73,13 @@ public boolean init(final Configuration conf, final Configuration oldConf) { this.oldPeers.clear(); this.quorum = this.oldQuorum = 0; int index = 0; - if (conf != null) { + if (conf != null && !conf.isEmpty()) { for (final PeerId peer : conf) { this.peers.add(new UnfoundPeerId(peer, index++, false)); } + quorum = conf.getQuorum().getW(); } - this.quorum = this.peers.size() / 2 + 1; if (oldConf == null) { return true; } @@ -80,7 +88,9 @@ public boolean init(final Configuration conf, final Configuration oldConf) { this.oldPeers.add(new UnfoundPeerId(peer, index++, false)); } - this.oldQuorum = this.oldPeers.size() / 2 + 1; + if (!oldConf.isEmpty()) { + this.oldQuorum = oldConf.getQuorum().getW(); + } return true; } @@ -138,4 +148,10 @@ public void grant(final PeerId peerId) { public boolean isGranted() { return this.quorum <= 0 && this.oldQuorum <= 0; } + + public void refreshBallot(Configuration conf, Configuration oldConf) { + if (!this.init(conf, oldConf)) { + LOG.error("An error occurred while refreshing the configuration"); + } + } } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/BallotFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/BallotFactory.java new file mode 100644 index 000000000..b7b016901 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/BallotFactory.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.entity; + +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.Objects; + +/** + * @author Akai + */ +public final class BallotFactory { + private static final Logger LOG = LoggerFactory.getLogger(BallotFactory.class); + private static final String defaultDecimalFactor = "0.1"; + private static final BigDecimal defaultDecimal = new BigDecimal(defaultDecimalFactor); + + public static Quorum buildFlexibleQuorum(Integer readFactor, Integer writeFactor, int size) { + // if size equals 0,config must be empty,so we just return null + if (size == 0) { + return null; + } + // Check if factors are valid + if (!checkValid(readFactor, writeFactor)) { + LOG.error("Invalid factor, factor's range must be (0,10) and the sum of factor should be 10"); + return null; + } + // Partial factor is empty + if (Objects.isNull(writeFactor)) { + writeFactor = 10 - readFactor; + } + if (Objects.isNull(readFactor)) { + readFactor = 10 - writeFactor; + } + // Calculate quorum + int w = calculateWriteQuorum(writeFactor, size); + int r = calculateReadQuorum(readFactor, size); + return new Quorum(w, r); + } + + public static Quorum buildMajorityQuorum(int size) { + // if size equals 0,config must be empty,so we just return null + if (size == 0) { + return null; + } + int majorityQuorum = calculateMajorityQuorum(size); + return new Quorum(majorityQuorum, majorityQuorum); + } + + private static int calculateWriteQuorum(int writeFactor, int n) { + BigDecimal writeFactorDecimal = defaultDecimal.multiply(new BigDecimal(writeFactor)) + .multiply(new BigDecimal(n)); + return writeFactorDecimal.setScale(0, RoundingMode.CEILING).intValue(); + } + + private static int calculateReadQuorum(int readFactor, int n) { + int writeQuorum = calculateWriteQuorum(10 - readFactor, n); + return n - writeQuorum + 1; + } + + private static int calculateMajorityQuorum(int n) { + return n / 2 + 1; + } + + public static boolean checkValid(Integer readFactor, Integer writeFactor) { + if (Objects.isNull(readFactor) || Objects.isNull(writeFactor)) { + LOG.error("When turning on flexible mode, Both of readFactor and writeFactor should not be null."); + return false; + } + if (readFactor + writeFactor == 10 && readFactor > 0 && readFactor < 10 && writeFactor > 0 && writeFactor < 10) { + return true; + } + LOG.error("Fail to set quorum_nwr because the sum of read_factor and write_factor is {} , not 10", + readFactor + writeFactor); + return false; + } + + public static LogEntry convertConfigToLogEntry(LogEntry logEntry, Configuration conf) { + if (Objects.isNull(logEntry)) { + logEntry = new LogEntry(); + } + logEntry.setEnableFlexible(false); + logEntry.setPeers(conf.listPeers()); + final LogOutter.Quorum.Builder quorumBuilder = LogOutter.Quorum.newBuilder(); + LogOutter.Quorum quorum = quorumBuilder.setR(conf.getQuorum().getR()).setW(conf.getQuorum().getW()).build(); + logEntry.setQuorum(quorum); + return logEntry; + } + + public static LogEntry convertOldConfigToLogOuterEntry(LogEntry logEntry, Configuration conf) { + if (Objects.isNull(logEntry)) { + logEntry = new LogEntry(); + } + logEntry.setEnableFlexible(false); + logEntry.setOldPeers(conf.listPeers()); + final LogOutter.Quorum.Builder quorumBuilder = LogOutter.Quorum.newBuilder(); + LogOutter.Quorum quorum = quorumBuilder.setR(conf.getQuorum().getR()).setW(conf.getQuorum().getW()).build(); + logEntry.setOldQuorum(quorum); + return logEntry; + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/LogEntry.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/LogEntry.java index da4d466f2..2e0174aee 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/LogEntry.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/LogEntry.java @@ -18,12 +18,14 @@ import java.nio.ByteBuffer; import java.util.List; +import java.util.Objects; import com.alipay.sofa.jraft.entity.codec.LogEntryDecoder; import com.alipay.sofa.jraft.entity.codec.LogEntryEncoder; import com.alipay.sofa.jraft.entity.codec.v1.LogEntryV1CodecFactory; import com.alipay.sofa.jraft.entity.codec.v1.V1Decoder; import com.alipay.sofa.jraft.entity.codec.v1.V1Encoder; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import com.alipay.sofa.jraft.util.CrcUtil; /** @@ -55,6 +57,20 @@ public class LogEntry implements Checksum { private long checksum; /** true when the log has checksum **/ private boolean hasChecksum; + /** read factor for flexible raft **/ + private Integer readFactor; + /** write factor for flexible raft **/ + private Integer writeFactor; + /** old read factor for flexible raft **/ + private Integer oldReadFactor; + /** old write factor for flexible raft **/ + private Integer oldWriteFactor; + /** enable flexible raft or not **/ + private Boolean isEnableFlexible; + /** quorum for log entry **/ + private LogOutter.Quorum quorum; + /** old quorum for log entry **/ + private LogOutter.Quorum oldQuorum; public List getLearners() { return this.learners; @@ -195,6 +211,70 @@ public void setOldPeers(final List oldPeers) { this.oldPeers = oldPeers; } + public Integer getReadFactor() { + return readFactor; + } + + public void setReadFactor(Integer readFactor) { + this.readFactor = readFactor; + } + + public Integer getWriteFactor() { + return writeFactor; + } + + public void setWriteFactor(Integer writeFactor) { + this.writeFactor = writeFactor; + } + + public Integer getOldReadFactor() { + return oldReadFactor; + } + + public void setOldReadFactor(Integer oldReadFactor) { + this.oldReadFactor = oldReadFactor; + } + + public Integer getOldWriteFactor() { + return oldWriteFactor; + } + + public void setOldWriteFactor(Integer oldWriteFactor) { + this.oldWriteFactor = oldWriteFactor; + } + + public Boolean getEnableFlexible() { + return isEnableFlexible; + } + + public void setEnableFlexible(Boolean enableFlexible) { + isEnableFlexible = enableFlexible; + } + + public void setQuorum(LogOutter.Quorum quorum) { + this.quorum = quorum; + } + + public LogOutter.Quorum getQuorum() { + return quorum; + } + + public void setOldQuorum(LogOutter.Quorum quorum) { + this.oldQuorum = quorum; + } + + public LogOutter.Quorum getOldQuorum() { + return oldQuorum; + } + + public boolean haveFactorValue() { + return Objects.nonNull(readFactor) || Objects.nonNull(writeFactor); + } + + public boolean haveOldFactorValue() { + return Objects.nonNull(oldReadFactor) || Objects.nonNull(oldWriteFactor); + } + /** * Returns the log data, it's not read-only, you SHOULD take care it's modification and * thread-safety by yourself. @@ -232,7 +312,9 @@ public void setData(final ByteBuffer data) { public String toString() { return "LogEntry [type=" + this.type + ", id=" + this.id + ", peers=" + this.peers + ", oldPeers=" + this.oldPeers + ", learners=" + this.learners + ", oldLearners=" + this.oldLearners + ", data=" - + (this.data != null ? this.data.remaining() : 0) + "]"; + + (this.data != null ? this.data.remaining() : 0) + ", readFactor=" + this.readFactor + ", writeFactor=" + + this.writeFactor + ", oldReadFactor=" + oldReadFactor + ", oldWriteFactor=" + oldWriteFactor + + ", quorum=" + quorum + ", oldQuorum=" + oldQuorum + ", isEnableFlexible=" + isEnableFlexible + "]"; } @Override diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/RaftOutter.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/RaftOutter.java index 4f39aa01e..22e03c38b 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/RaftOutter.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/RaftOutter.java @@ -181,6 +181,86 @@ public interface EntryMetaOrBuilder extends * repeated string old_learners = 8; */ com.google.protobuf.ByteString getOldLearnersBytes(int index); + + /** + * optional int32 read_factor = 9; + */ + boolean hasReadFactor(); + + /** + * optional int32 read_factor = 9; + */ + int getReadFactor(); + + /** + * optional int32 write_factor = 10; + */ + boolean hasWriteFactor(); + + /** + * optional int32 write_factor = 10; + */ + int getWriteFactor(); + + /** + * optional int32 old_read_factor = 11; + */ + boolean hasOldReadFactor(); + + /** + * optional int32 old_read_factor = 11; + */ + int getOldReadFactor(); + + /** + * optional int32 old_write_factor = 12; + */ + boolean hasOldWriteFactor(); + + /** + * optional int32 old_write_factor = 12; + */ + int getOldWriteFactor(); + + /** + * optional bool isEnableFlexible = 13; + */ + boolean hasIsEnableFlexible(); + + /** + * optional bool isEnableFlexible = 13; + */ + boolean getIsEnableFlexible(); + + /** + * optional .jraft.Quorum quorum = 14; + */ + boolean hasQuorum(); + + /** + * optional .jraft.Quorum quorum = 14; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum(); + + /** + * optional .jraft.Quorum quorum = 14; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder(); + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + boolean hasOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder(); } /** @@ -205,6 +285,11 @@ private EntryMeta() { checksum_ = 0L; learners_ = com.google.protobuf.LazyStringArrayList.EMPTY; oldLearners_ = com.google.protobuf.LazyStringArrayList.EMPTY; + readFactor_ = 0; + writeFactor_ = 0; + oldReadFactor_ = 0; + oldWriteFactor_ = 0; + isEnableFlexible_ = false; } @java.lang.Override @@ -299,6 +384,59 @@ private EntryMeta(com.google.protobuf.CodedInputStream input, oldLearners_.add(bs); break; } + case 72: { + bitField0_ |= 0x00000010; + readFactor_ = input.readInt32(); + break; + } + case 80: { + bitField0_ |= 0x00000020; + writeFactor_ = input.readInt32(); + break; + } + case 88: { + bitField0_ |= 0x00000040; + oldReadFactor_ = input.readInt32(); + break; + } + case 96: { + bitField0_ |= 0x00000080; + oldWriteFactor_ = input.readInt32(); + break; + } + case 104: { + bitField0_ |= 0x00000100; + isEnableFlexible_ = input.readBool(); + break; + } + case 114: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000200) == 0x00000200)) { + subBuilder = quorum_.toBuilder(); + } + quorum_ = input.readMessage(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(quorum_); + quorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000200; + break; + } + case 122: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000400) == 0x00000400)) { + subBuilder = oldQuorum_.toBuilder(); + } + oldQuorum_ = input.readMessage( + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(oldQuorum_); + oldQuorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000400; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -556,6 +694,143 @@ public com.google.protobuf.ByteString getOldLearnersBytes(int index) { return oldLearners_.getByteString(index); } + public static final int READ_FACTOR_FIELD_NUMBER = 9; + private int readFactor_; + + /** + * optional int32 read_factor = 9; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + + /** + * optional int32 read_factor = 9; + */ + public int getReadFactor() { + return readFactor_; + } + + public static final int WRITE_FACTOR_FIELD_NUMBER = 10; + private int writeFactor_; + + /** + * optional int32 write_factor = 10; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + + /** + * optional int32 write_factor = 10; + */ + public int getWriteFactor() { + return writeFactor_; + } + + public static final int OLD_READ_FACTOR_FIELD_NUMBER = 11; + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 11; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional int32 old_read_factor = 11; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + public static final int OLD_WRITE_FACTOR_FIELD_NUMBER = 12; + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 12; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional int32 old_write_factor = 12; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + public static final int ISENABLEFLEXIBLE_FIELD_NUMBER = 13; + private boolean isEnableFlexible_; + + /** + * optional bool isEnableFlexible = 13; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional bool isEnableFlexible = 13; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + public static final int QUORUM_FIELD_NUMBER = 14; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_; + + /** + * optional .jraft.Quorum quorum = 14; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + public static final int OLD_QUORUM_FIELD_NUMBER = 15; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_; + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -602,6 +877,27 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io for (int i = 0; i < oldLearners_.size(); i++) { com.google.protobuf.GeneratedMessageV3.writeString(output, 8, oldLearners_.getRaw(i)); } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeInt32(9, readFactor_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeInt32(10, writeFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeInt32(11, oldReadFactor_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeInt32(12, oldWriteFactor_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeBool(13, isEnableFlexible_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeMessage(14, getQuorum()); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeMessage(15, getOldQuorum()); + } unknownFields.writeTo(output); } @@ -655,6 +951,27 @@ public int getSerializedSize() { size += dataSize; size += 1 * getOldLearnersList().size(); } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(9, readFactor_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(10, writeFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(11, oldReadFactor_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(12, oldWriteFactor_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream.computeBoolSize(13, isEnableFlexible_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(14, getQuorum()); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(15, getOldQuorum()); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -691,6 +1008,34 @@ public boolean equals(final java.lang.Object obj) { } result = result && getLearnersList().equals(other.getLearnersList()); result = result && getOldLearnersList().equals(other.getOldLearnersList()); + result = result && (hasReadFactor() == other.hasReadFactor()); + if (hasReadFactor()) { + result = result && (getReadFactor() == other.getReadFactor()); + } + result = result && (hasWriteFactor() == other.hasWriteFactor()); + if (hasWriteFactor()) { + result = result && (getWriteFactor() == other.getWriteFactor()); + } + result = result && (hasOldReadFactor() == other.hasOldReadFactor()); + if (hasOldReadFactor()) { + result = result && (getOldReadFactor() == other.getOldReadFactor()); + } + result = result && (hasOldWriteFactor() == other.hasOldWriteFactor()); + if (hasOldWriteFactor()) { + result = result && (getOldWriteFactor() == other.getOldWriteFactor()); + } + result = result && (hasIsEnableFlexible() == other.hasIsEnableFlexible()); + if (hasIsEnableFlexible()) { + result = result && (getIsEnableFlexible() == other.getIsEnableFlexible()); + } + result = result && (hasQuorum() == other.hasQuorum()); + if (hasQuorum()) { + result = result && getQuorum().equals(other.getQuorum()); + } + result = result && (hasOldQuorum() == other.hasOldQuorum()); + if (hasOldQuorum()) { + result = result && getOldQuorum().equals(other.getOldQuorum()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -734,6 +1079,34 @@ public int hashCode() { hash = (37 * hash) + OLD_LEARNERS_FIELD_NUMBER; hash = (53 * hash) + getOldLearnersList().hashCode(); } + if (hasReadFactor()) { + hash = (37 * hash) + READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getReadFactor(); + } + if (hasWriteFactor()) { + hash = (37 * hash) + WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getWriteFactor(); + } + if (hasOldReadFactor()) { + hash = (37 * hash) + OLD_READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldReadFactor(); + } + if (hasOldWriteFactor()) { + hash = (37 * hash) + OLD_WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldWriteFactor(); + } + if (hasIsEnableFlexible()) { + hash = (37 * hash) + ISENABLEFLEXIBLE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getIsEnableFlexible()); + } + if (hasQuorum()) { + hash = (37 * hash) + QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getQuorum().hashCode(); + } + if (hasOldQuorum()) { + hash = (37 * hash) + OLD_QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getOldQuorum().hashCode(); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -857,6 +1230,8 @@ private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getQuorumFieldBuilder(); + getOldQuorumFieldBuilder(); } } @@ -878,6 +1253,28 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000040); oldLearners_ = com.google.protobuf.LazyStringArrayList.EMPTY; bitField0_ = (bitField0_ & ~0x00000080); + readFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000100); + writeFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000200); + oldReadFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000400); + oldWriteFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000800); + isEnableFlexible_ = false; + bitField0_ = (bitField0_ & ~0x00001000); + if (quorumBuilder_ == null) { + quorum_ = null; + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00002000); + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00004000); return this; } @@ -938,6 +1335,42 @@ public com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta buildPartial() { bitField0_ = (bitField0_ & ~0x00000080); } result.oldLearners_ = oldLearners_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000010; + } + result.readFactor_ = readFactor_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000020; + } + result.writeFactor_ = writeFactor_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000040; + } + result.oldReadFactor_ = oldReadFactor_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000080; + } + result.oldWriteFactor_ = oldWriteFactor_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000100; + } + result.isEnableFlexible_ = isEnableFlexible_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00000200; + } + if (quorumBuilder_ == null) { + result.quorum_ = quorum_; + } else { + result.quorum_ = quorumBuilder_.build(); + } + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00000400; + } + if (oldQuorumBuilder_ == null) { + result.oldQuorum_ = oldQuorum_; + } else { + result.oldQuorum_ = oldQuorumBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1033,6 +1466,27 @@ public Builder mergeFrom(com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta other } onChanged(); } + if (other.hasReadFactor()) { + setReadFactor(other.getReadFactor()); + } + if (other.hasWriteFactor()) { + setWriteFactor(other.getWriteFactor()); + } + if (other.hasOldReadFactor()) { + setOldReadFactor(other.getOldReadFactor()); + } + if (other.hasOldWriteFactor()) { + setOldWriteFactor(other.getOldWriteFactor()); + } + if (other.hasIsEnableFlexible()) { + setIsEnableFlexible(other.getIsEnableFlexible()); + } + if (other.hasQuorum()) { + mergeQuorum(other.getQuorum()); + } + if (other.hasOldQuorum()) { + mergeOldQuorum(other.getOldQuorum()); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -1661,55 +2115,475 @@ public Builder addOldLearnersBytes(com.google.protobuf.ByteString value) { return this; } - public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { - return super.setUnknownFields(unknownFields); - } + private int readFactor_; - public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { - return super.mergeUnknownFields(unknownFields); + /** + * optional int32 read_factor = 9; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000100) == 0x00000100); } - // @@protoc_insertion_point(builder_scope:jraft.EntryMeta) - } + /** + * optional int32 read_factor = 9; + */ + public int getReadFactor() { + return readFactor_; + } - // @@protoc_insertion_point(class_scope:jraft.EntryMeta) - private static final com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta DEFAULT_INSTANCE; - static { - DEFAULT_INSTANCE = new com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta(); - } + /** + * optional int32 read_factor = 9; + */ + public Builder setReadFactor(int value) { + bitField0_ |= 0x00000100; + readFactor_ = value; + onChanged(); + return this; + } - public static com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta getDefaultInstance() { - return DEFAULT_INSTANCE; - } + /** + * optional int32 read_factor = 9; + */ + public Builder clearReadFactor() { + bitField0_ = (bitField0_ & ~0x00000100); + readFactor_ = 0; + onChanged(); + return this; + } - @java.lang.Deprecated - public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { - public EntryMeta parsePartialFrom(com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return new EntryMeta(input, - extensionRegistry); - } - }; + private int writeFactor_; - public static com.google.protobuf.Parser parser() { - return PARSER; - } + /** + * optional int32 write_factor = 10; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } - @java.lang.Override - public com.google.protobuf.Parser getParserForType() { - return PARSER; - } + /** + * optional int32 write_factor = 10; + */ + public int getWriteFactor() { + return writeFactor_; + } - public com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta getDefaultInstanceForType() { - return DEFAULT_INSTANCE; - } + /** + * optional int32 write_factor = 10; + */ + public Builder setWriteFactor(int value) { + bitField0_ |= 0x00000200; + writeFactor_ = value; + onChanged(); + return this; + } - } + /** + * optional int32 write_factor = 10; + */ + public Builder clearWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000200); + writeFactor_ = 0; + onChanged(); + return this; + } - public interface SnapshotMetaOrBuilder extends - // @@protoc_insertion_point(interface_extends:jraft.SnapshotMeta) - com.google.protobuf.MessageOrBuilder { + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 11; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + + /** + * optional int32 old_read_factor = 11; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + /** + * optional int32 old_read_factor = 11; + */ + public Builder setOldReadFactor(int value) { + bitField0_ |= 0x00000400; + oldReadFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_read_factor = 11; + */ + public Builder clearOldReadFactor() { + bitField0_ = (bitField0_ & ~0x00000400); + oldReadFactor_ = 0; + onChanged(); + return this; + } + + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 12; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + + /** + * optional int32 old_write_factor = 12; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + /** + * optional int32 old_write_factor = 12; + */ + public Builder setOldWriteFactor(int value) { + bitField0_ |= 0x00000800; + oldWriteFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_write_factor = 12; + */ + public Builder clearOldWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000800); + oldWriteFactor_ = 0; + onChanged(); + return this; + } + + private boolean isEnableFlexible_; + + /** + * optional bool isEnableFlexible = 13; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + + /** + * optional bool isEnableFlexible = 13; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + /** + * optional bool isEnableFlexible = 13; + */ + public Builder setIsEnableFlexible(boolean value) { + bitField0_ |= 0x00001000; + isEnableFlexible_ = value; + onChanged(); + return this; + } + + /** + * optional bool isEnableFlexible = 13; + */ + public Builder clearIsEnableFlexible() { + bitField0_ = (bitField0_ & ~0x00001000); + isEnableFlexible_ = false; + onChanged(); + return this; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 quorumBuilder_; + + /** + * optional .jraft.Quorum quorum = 14; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + if (quorumBuilder_ == null) { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } else { + return quorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + quorum_ = value; + onChanged(); + } else { + quorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00002000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (quorumBuilder_ == null) { + quorum_ = builderForValue.build(); + onChanged(); + } else { + quorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00002000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public Builder mergeQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (((bitField0_ & 0x00002000) == 0x00002000) && quorum_ != null + && quorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + quorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(quorum_) + .mergeFrom(value).buildPartial(); + } else { + quorum_ = value; + } + onChanged(); + } else { + quorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00002000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public Builder clearQuorum() { + if (quorumBuilder_ == null) { + quorum_ = null; + onChanged(); + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00002000); + return this; + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getQuorumBuilder() { + bitField0_ |= 0x00002000; + onChanged(); + return getQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + if (quorumBuilder_ != null) { + return quorumBuilder_.getMessageOrBuilder(); + } else { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } + } + + /** + * optional .jraft.Quorum quorum = 14; + */ + private com.google.protobuf.SingleFieldBuilderV3 getQuorumFieldBuilder() { + if (quorumBuilder_ == null) { + quorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getQuorum(), getParentForChildren(), isClean()); + quorum_ = null; + } + return quorumBuilder_; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 oldQuorumBuilder_; + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + if (oldQuorumBuilder_ == null) { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } else { + return oldQuorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + oldQuorum_ = value; + onChanged(); + } else { + oldQuorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = builderForValue.build(); + onChanged(); + } else { + oldQuorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public Builder mergeOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (((bitField0_ & 0x00004000) == 0x00004000) && oldQuorum_ != null + && oldQuorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + oldQuorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(oldQuorum_) + .mergeFrom(value).buildPartial(); + } else { + oldQuorum_ = value; + } + onChanged(); + } else { + oldQuorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public Builder clearOldQuorum() { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + onChanged(); + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00004000); + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getOldQuorumBuilder() { + bitField0_ |= 0x00004000; + onChanged(); + return getOldQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + if (oldQuorumBuilder_ != null) { + return oldQuorumBuilder_.getMessageOrBuilder(); + } else { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } + } + + /** + * optional .jraft.Quorum old_quorum = 15; + */ + private com.google.protobuf.SingleFieldBuilderV3 getOldQuorumFieldBuilder() { + if (oldQuorumBuilder_ == null) { + oldQuorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getOldQuorum(), getParentForChildren(), isClean()); + oldQuorum_ = null; + } + return oldQuorumBuilder_; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.EntryMeta) + } + + // @@protoc_insertion_point(class_scope:jraft.EntryMeta) + private static final com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta(); + } + + public static com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public EntryMeta parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new EntryMeta(input, + extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.entity.RaftOutter.EntryMeta getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface SnapshotMetaOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.SnapshotMeta) + com.google.protobuf.MessageOrBuilder { /** * required int64 last_included_index = 1; @@ -1810,6 +2684,86 @@ public interface SnapshotMetaOrBuilder extends * repeated string old_learners = 6; */ com.google.protobuf.ByteString getOldLearnersBytes(int index); + + /** + * optional int32 read_factor = 7; + */ + boolean hasReadFactor(); + + /** + * optional int32 read_factor = 7; + */ + int getReadFactor(); + + /** + * optional int32 write_factor = 8; + */ + boolean hasWriteFactor(); + + /** + * optional int32 write_factor = 8; + */ + int getWriteFactor(); + + /** + * optional int32 old_read_factor = 9; + */ + boolean hasOldReadFactor(); + + /** + * optional int32 old_read_factor = 9; + */ + int getOldReadFactor(); + + /** + * optional int32 old_write_factor = 10; + */ + boolean hasOldWriteFactor(); + + /** + * optional int32 old_write_factor = 10; + */ + int getOldWriteFactor(); + + /** + * optional bool isEnableFlexible = 11; + */ + boolean hasIsEnableFlexible(); + + /** + * optional bool isEnableFlexible = 11; + */ + boolean getIsEnableFlexible(); + + /** + * optional .jraft.Quorum quorum = 12; + */ + boolean hasQuorum(); + + /** + * optional .jraft.Quorum quorum = 12; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum(); + + /** + * optional .jraft.Quorum quorum = 12; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder(); + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + boolean hasOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder(); } /** @@ -1832,6 +2786,11 @@ private SnapshotMeta() { oldPeers_ = com.google.protobuf.LazyStringArrayList.EMPTY; learners_ = com.google.protobuf.LazyStringArrayList.EMPTY; oldLearners_ = com.google.protobuf.LazyStringArrayList.EMPTY; + readFactor_ = 0; + writeFactor_ = 0; + oldReadFactor_ = 0; + oldWriteFactor_ = 0; + isEnableFlexible_ = false; } @java.lang.Override @@ -1909,6 +2868,59 @@ private SnapshotMeta(com.google.protobuf.CodedInputStream input, oldLearners_.add(bs); break; } + case 56: { + bitField0_ |= 0x00000004; + readFactor_ = input.readInt32(); + break; + } + case 64: { + bitField0_ |= 0x00000008; + writeFactor_ = input.readInt32(); + break; + } + case 72: { + bitField0_ |= 0x00000010; + oldReadFactor_ = input.readInt32(); + break; + } + case 80: { + bitField0_ |= 0x00000020; + oldWriteFactor_ = input.readInt32(); + break; + } + case 88: { + bitField0_ |= 0x00000040; + isEnableFlexible_ = input.readBool(); + break; + } + case 98: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000080) == 0x00000080)) { + subBuilder = quorum_.toBuilder(); + } + quorum_ = input.readMessage(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(quorum_); + quorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000080; + break; + } + case 106: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000100) == 0x00000100)) { + subBuilder = oldQuorum_.toBuilder(); + } + oldQuorum_ = input.readMessage( + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(oldQuorum_); + oldQuorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000100; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -2102,6 +3114,143 @@ public com.google.protobuf.ByteString getOldLearnersBytes(int index) { return oldLearners_.getByteString(index); } + public static final int READ_FACTOR_FIELD_NUMBER = 7; + private int readFactor_; + + /** + * optional int32 read_factor = 7; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional int32 read_factor = 7; + */ + public int getReadFactor() { + return readFactor_; + } + + public static final int WRITE_FACTOR_FIELD_NUMBER = 8; + private int writeFactor_; + + /** + * optional int32 write_factor = 8; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional int32 write_factor = 8; + */ + public int getWriteFactor() { + return writeFactor_; + } + + public static final int OLD_READ_FACTOR_FIELD_NUMBER = 9; + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 9; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + + /** + * optional int32 old_read_factor = 9; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + public static final int OLD_WRITE_FACTOR_FIELD_NUMBER = 10; + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 10; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + + /** + * optional int32 old_write_factor = 10; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + public static final int ISENABLEFLEXIBLE_FIELD_NUMBER = 11; + private boolean isEnableFlexible_; + + /** + * optional bool isEnableFlexible = 11; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional bool isEnableFlexible = 11; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + public static final int QUORUM_FIELD_NUMBER = 12; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_; + + /** + * optional .jraft.Quorum quorum = 12; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + public static final int OLD_QUORUM_FIELD_NUMBER = 13; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_; + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -2142,6 +3291,27 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io for (int i = 0; i < oldLearners_.size(); i++) { com.google.protobuf.GeneratedMessageV3.writeString(output, 6, oldLearners_.getRaw(i)); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(7, readFactor_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(8, writeFactor_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeInt32(9, oldReadFactor_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeInt32(10, oldWriteFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBool(11, isEnableFlexible_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeMessage(12, getQuorum()); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeMessage(13, getOldQuorum()); + } unknownFields.writeTo(output); } @@ -2189,6 +3359,27 @@ public int getSerializedSize() { size += dataSize; size += 1 * getOldLearnersList().size(); } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(7, readFactor_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(8, writeFactor_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(9, oldReadFactor_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(10, oldWriteFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream.computeBoolSize(11, isEnableFlexible_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(12, getQuorum()); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(13, getOldQuorum()); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -2217,6 +3408,34 @@ public boolean equals(final java.lang.Object obj) { result = result && getOldPeersList().equals(other.getOldPeersList()); result = result && getLearnersList().equals(other.getLearnersList()); result = result && getOldLearnersList().equals(other.getOldLearnersList()); + result = result && (hasReadFactor() == other.hasReadFactor()); + if (hasReadFactor()) { + result = result && (getReadFactor() == other.getReadFactor()); + } + result = result && (hasWriteFactor() == other.hasWriteFactor()); + if (hasWriteFactor()) { + result = result && (getWriteFactor() == other.getWriteFactor()); + } + result = result && (hasOldReadFactor() == other.hasOldReadFactor()); + if (hasOldReadFactor()) { + result = result && (getOldReadFactor() == other.getOldReadFactor()); + } + result = result && (hasOldWriteFactor() == other.hasOldWriteFactor()); + if (hasOldWriteFactor()) { + result = result && (getOldWriteFactor() == other.getOldWriteFactor()); + } + result = result && (hasIsEnableFlexible() == other.hasIsEnableFlexible()); + if (hasIsEnableFlexible()) { + result = result && (getIsEnableFlexible() == other.getIsEnableFlexible()); + } + result = result && (hasQuorum() == other.hasQuorum()); + if (hasQuorum()) { + result = result && getQuorum().equals(other.getQuorum()); + } + result = result && (hasOldQuorum() == other.hasOldQuorum()); + if (hasOldQuorum()) { + result = result && getOldQuorum().equals(other.getOldQuorum()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -2252,6 +3471,34 @@ public int hashCode() { hash = (37 * hash) + OLD_LEARNERS_FIELD_NUMBER; hash = (53 * hash) + getOldLearnersList().hashCode(); } + if (hasReadFactor()) { + hash = (37 * hash) + READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getReadFactor(); + } + if (hasWriteFactor()) { + hash = (37 * hash) + WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getWriteFactor(); + } + if (hasOldReadFactor()) { + hash = (37 * hash) + OLD_READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldReadFactor(); + } + if (hasOldWriteFactor()) { + hash = (37 * hash) + OLD_WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldWriteFactor(); + } + if (hasIsEnableFlexible()) { + hash = (37 * hash) + ISENABLEFLEXIBLE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getIsEnableFlexible()); + } + if (hasQuorum()) { + hash = (37 * hash) + QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getQuorum().hashCode(); + } + if (hasOldQuorum()) { + hash = (37 * hash) + OLD_QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getOldQuorum().hashCode(); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -2375,6 +3622,8 @@ private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getQuorumFieldBuilder(); + getOldQuorumFieldBuilder(); } } @@ -2392,6 +3641,28 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000010); oldLearners_ = com.google.protobuf.LazyStringArrayList.EMPTY; bitField0_ = (bitField0_ & ~0x00000020); + readFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000040); + writeFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000080); + oldReadFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000100); + oldWriteFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000200); + isEnableFlexible_ = false; + bitField0_ = (bitField0_ & ~0x00000400); + if (quorumBuilder_ == null) { + quorum_ = null; + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000800); + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00001000); return this; } @@ -2444,6 +3715,42 @@ public com.alipay.sofa.jraft.entity.RaftOutter.SnapshotMeta buildPartial() { bitField0_ = (bitField0_ & ~0x00000020); } result.oldLearners_ = oldLearners_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000004; + } + result.readFactor_ = readFactor_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000008; + } + result.writeFactor_ = writeFactor_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000010; + } + result.oldReadFactor_ = oldReadFactor_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000020; + } + result.oldWriteFactor_ = oldWriteFactor_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000040; + } + result.isEnableFlexible_ = isEnableFlexible_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000080; + } + if (quorumBuilder_ == null) { + result.quorum_ = quorum_; + } else { + result.quorum_ = quorumBuilder_.build(); + } + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000100; + } + if (oldQuorumBuilder_ == null) { + result.oldQuorum_ = oldQuorum_; + } else { + result.oldQuorum_ = oldQuorumBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -2533,6 +3840,27 @@ public Builder mergeFrom(com.alipay.sofa.jraft.entity.RaftOutter.SnapshotMeta ot } onChanged(); } + if (other.hasReadFactor()) { + setReadFactor(other.getReadFactor()); + } + if (other.hasWriteFactor()) { + setWriteFactor(other.getWriteFactor()); + } + if (other.hasOldReadFactor()) { + setOldReadFactor(other.getOldReadFactor()); + } + if (other.hasOldWriteFactor()) { + setOldWriteFactor(other.getOldWriteFactor()); + } + if (other.hasIsEnableFlexible()) { + setIsEnableFlexible(other.getIsEnableFlexible()); + } + if (other.hasQuorum()) { + mergeQuorum(other.getQuorum()); + } + if (other.hasOldQuorum()) { + mergeOldQuorum(other.getOldQuorum()); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -3023,6 +4351,426 @@ public Builder addOldLearnersBytes(com.google.protobuf.ByteString value) { return this; } + private int readFactor_; + + /** + * optional int32 read_factor = 7; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional int32 read_factor = 7; + */ + public int getReadFactor() { + return readFactor_; + } + + /** + * optional int32 read_factor = 7; + */ + public Builder setReadFactor(int value) { + bitField0_ |= 0x00000040; + readFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 read_factor = 7; + */ + public Builder clearReadFactor() { + bitField0_ = (bitField0_ & ~0x00000040); + readFactor_ = 0; + onChanged(); + return this; + } + + private int writeFactor_; + + /** + * optional int32 write_factor = 8; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional int32 write_factor = 8; + */ + public int getWriteFactor() { + return writeFactor_; + } + + /** + * optional int32 write_factor = 8; + */ + public Builder setWriteFactor(int value) { + bitField0_ |= 0x00000080; + writeFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 write_factor = 8; + */ + public Builder clearWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000080); + writeFactor_ = 0; + onChanged(); + return this; + } + + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 9; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional int32 old_read_factor = 9; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + /** + * optional int32 old_read_factor = 9; + */ + public Builder setOldReadFactor(int value) { + bitField0_ |= 0x00000100; + oldReadFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_read_factor = 9; + */ + public Builder clearOldReadFactor() { + bitField0_ = (bitField0_ & ~0x00000100); + oldReadFactor_ = 0; + onChanged(); + return this; + } + + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 10; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional int32 old_write_factor = 10; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + /** + * optional int32 old_write_factor = 10; + */ + public Builder setOldWriteFactor(int value) { + bitField0_ |= 0x00000200; + oldWriteFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_write_factor = 10; + */ + public Builder clearOldWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000200); + oldWriteFactor_ = 0; + onChanged(); + return this; + } + + private boolean isEnableFlexible_; + + /** + * optional bool isEnableFlexible = 11; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + + /** + * optional bool isEnableFlexible = 11; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + /** + * optional bool isEnableFlexible = 11; + */ + public Builder setIsEnableFlexible(boolean value) { + bitField0_ |= 0x00000400; + isEnableFlexible_ = value; + onChanged(); + return this; + } + + /** + * optional bool isEnableFlexible = 11; + */ + public Builder clearIsEnableFlexible() { + bitField0_ = (bitField0_ & ~0x00000400); + isEnableFlexible_ = false; + onChanged(); + return this; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 quorumBuilder_; + + /** + * optional .jraft.Quorum quorum = 12; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + if (quorumBuilder_ == null) { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } else { + return quorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + quorum_ = value; + onChanged(); + } else { + quorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00000800; + return this; + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (quorumBuilder_ == null) { + quorum_ = builderForValue.build(); + onChanged(); + } else { + quorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000800; + return this; + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public Builder mergeQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (((bitField0_ & 0x00000800) == 0x00000800) && quorum_ != null + && quorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + quorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(quorum_) + .mergeFrom(value).buildPartial(); + } else { + quorum_ = value; + } + onChanged(); + } else { + quorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000800; + return this; + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public Builder clearQuorum() { + if (quorumBuilder_ == null) { + quorum_ = null; + onChanged(); + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000800); + return this; + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getQuorumBuilder() { + bitField0_ |= 0x00000800; + onChanged(); + return getQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + if (quorumBuilder_ != null) { + return quorumBuilder_.getMessageOrBuilder(); + } else { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } + } + + /** + * optional .jraft.Quorum quorum = 12; + */ + private com.google.protobuf.SingleFieldBuilderV3 getQuorumFieldBuilder() { + if (quorumBuilder_ == null) { + quorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getQuorum(), getParentForChildren(), isClean()); + quorum_ = null; + } + return quorumBuilder_; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 oldQuorumBuilder_; + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + if (oldQuorumBuilder_ == null) { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } else { + return oldQuorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + oldQuorum_ = value; + onChanged(); + } else { + oldQuorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00001000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = builderForValue.build(); + onChanged(); + } else { + oldQuorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00001000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public Builder mergeOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (((bitField0_ & 0x00001000) == 0x00001000) && oldQuorum_ != null + && oldQuorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + oldQuorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(oldQuorum_) + .mergeFrom(value).buildPartial(); + } else { + oldQuorum_ = value; + } + onChanged(); + } else { + oldQuorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00001000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public Builder clearOldQuorum() { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + onChanged(); + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00001000); + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getOldQuorumBuilder() { + bitField0_ |= 0x00001000; + onChanged(); + return getOldQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + if (oldQuorumBuilder_ != null) { + return oldQuorumBuilder_.getMessageOrBuilder(); + } else { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } + } + + /** + * optional .jraft.Quorum old_quorum = 13; + */ + private com.google.protobuf.SingleFieldBuilderV3 getOldQuorumFieldBuilder() { + if (oldQuorumBuilder_ == null) { + oldQuorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getOldQuorum(), getParentForChildren(), isClean()); + oldQuorum_ = null; + } + return oldQuorumBuilder_; + } + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } @@ -3080,34 +4828,48 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { private static com.google.protobuf.Descriptors.FileDescriptor descriptor; static { - java.lang.String[] descriptorData = { "\n\nraft.proto\022\005jraft\032\nenum.proto\"\247\001\n\tEntr" - + "yMeta\022\014\n\004term\030\001 \002(\003\022\036\n\004type\030\002 \002(\0162\020.jraf" - + "t.EntryType\022\r\n\005peers\030\003 \003(\t\022\020\n\010data_len\030\004" - + " \001(\003\022\021\n\told_peers\030\005 \003(\t\022\020\n\010checksum\030\006 \001(" - + "\003\022\020\n\010learners\030\007 \003(\t\022\024\n\014old_learners\030\010 \003(" - + "\t\"\221\001\n\014SnapshotMeta\022\033\n\023last_included_inde" - + "x\030\001 \002(\003\022\032\n\022last_included_term\030\002 \002(\003\022\r\n\005p" - + "eers\030\003 \003(\t\022\021\n\told_peers\030\004 \003(\t\022\020\n\010learner" - + "s\030\005 \003(\t\022\024\n\014old_learners\030\006 \003(\tB*\n\034com.ali" - + "pay.sofa.jraft.entityB\nRaftOutter" }; + java.lang.String[] descriptorData = { "\n\nraft.proto\022\005jraft\032\nenum.proto\032\tlog.pro" + + "to\"\341\002\n\tEntryMeta\022\014\n\004term\030\001 \002(\003\022\036\n\004type\030\002" + + " \002(\0162\020.jraft.EntryType\022\r\n\005peers\030\003 \003(\t\022\020\n" + + "\010data_len\030\004 \001(\003\022\021\n\told_peers\030\005 \003(\t\022\020\n\010ch" + + "ecksum\030\006 \001(\003\022\020\n\010learners\030\007 \003(\t\022\024\n\014old_le" + + "arners\030\010 \003(\t\022\023\n\013read_factor\030\t \001(\005\022\024\n\014wri" + + "te_factor\030\n \001(\005\022\027\n\017old_read_factor\030\013 \001(\005" + + "\022\030\n\020old_write_factor\030\014 \001(\005\022\030\n\020isEnableFl" + + "exible\030\r \001(\010\022\035\n\006quorum\030\016 \001(\0132\r.jraft.Quo" + + "rum\022!\n\nold_quorum\030\017 \001(\0132\r.jraft.Quorum\"\313" + + "\002\n\014SnapshotMeta\022\033\n\023last_included_index\030\001" + + " \002(\003\022\032\n\022last_included_term\030\002 \002(\003\022\r\n\005peer" + + "s\030\003 \003(\t\022\021\n\told_peers\030\004 \003(\t\022\020\n\010learners\030\005" + + " \003(\t\022\024\n\014old_learners\030\006 \003(\t\022\023\n\013read_facto" + + "r\030\007 \001(\005\022\024\n\014write_factor\030\010 \001(\005\022\027\n\017old_rea" + + "d_factor\030\t \001(\005\022\030\n\020old_write_factor\030\n \001(\005" + + "\022\030\n\020isEnableFlexible\030\013 \001(\010\022\035\n\006quorum\030\014 \001" + + "(\0132\r.jraft.Quorum\022!\n\nold_quorum\030\r \001(\0132\r." + + "jraft.QuorumB*\n\034com.alipay.sofa.jraft.en" + "tityB\nRaftOutter" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; return null; } }; - com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { com.alipay.sofa.jraft.entity.EnumOutter - .getDescriptor(), }, assigner); + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + com.alipay.sofa.jraft.entity.EnumOutter.getDescriptor(), + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.getDescriptor(), }, assigner); internal_static_jraft_EntryMeta_descriptor = getDescriptor().getMessageTypes().get(0); internal_static_jraft_EntryMeta_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_jraft_EntryMeta_descriptor, new java.lang.String[] { "Term", "Type", "Peers", "DataLen", - "OldPeers", "Checksum", "Learners", "OldLearners", }); + "OldPeers", "Checksum", "Learners", "OldLearners", "ReadFactor", "WriteFactor", "OldReadFactor", + "OldWriteFactor", "IsEnableFlexible", "Quorum", "OldQuorum", }); internal_static_jraft_SnapshotMeta_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_jraft_SnapshotMeta_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_jraft_SnapshotMeta_descriptor, new java.lang.String[] { "LastIncludedIndex", - "LastIncludedTerm", "Peers", "OldPeers", "Learners", "OldLearners", }); + "LastIncludedTerm", "Peers", "OldPeers", "Learners", "OldLearners", "ReadFactor", "WriteFactor", + "OldReadFactor", "OldWriteFactor", "IsEnableFlexible", "Quorum", "OldQuorum", }); com.alipay.sofa.jraft.entity.EnumOutter.getDescriptor(); + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.getDescriptor(); } // @@protoc_insertion_point(outer_class_scope) diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Decoder.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Decoder.java index 20b5cc5db..e4cf36ac2 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Decoder.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Decoder.java @@ -25,6 +25,7 @@ import com.alipay.sofa.jraft.entity.LogId; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.entity.codec.LogEntryDecoder; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import com.alipay.sofa.jraft.util.AsciiStringUtil; import com.alipay.sofa.jraft.util.Bits; import com.alipay.sofa.jraft.util.BufferUtils; @@ -105,12 +106,56 @@ public void decode(final LogEntry log, final byte[] content) { } // data - if (content.length > pos) { - final int len = content.length - pos; - ByteBuffer data = ByteBuffer.allocate(len); - data.put(content, pos, len); + final int dataLength = Bits.getInt(content, pos); + pos += 4; + if (dataLength != 0) { + ByteBuffer data = ByteBuffer.allocate(dataLength); + data.put(content, pos, dataLength); BufferUtils.flip(data); log.setData(data); + pos += dataLength; + } + // isEnableFlexible + boolean isEnableFlexible = Bits.getBoolean(content, pos) == 1; + log.setEnableFlexible(isEnableFlexible); + pos += 1; + // quorum + boolean quorumExist = Bits.getBoolean(content, pos) == 1; + pos += 1; + if (quorumExist) { + int r = Bits.getInt(content, pos); + int w = Bits.getInt(content, pos + 4); + log.setQuorum(LogOutter.Quorum.newBuilder().setR(r).setW(w).build()); + pos += 8; + } + // oldQuorum + boolean oldQuorumExist = Bits.getBoolean(content, pos) == 1; + pos += 1; + if (oldQuorumExist) { + int r = Bits.getInt(content, pos); + int w = Bits.getInt(content, pos + 4); + log.setOldQuorum(LogOutter.Quorum.newBuilder().setR(r).setW(w).build()); + pos += 8; + } + // factor + boolean factorExist = Bits.getBoolean(content, pos) == 1; + pos += 1; + if (factorExist) { + int readFactor = Bits.getInt(content, pos); + int writeFactor = Bits.getInt(content, pos + 4); + log.setReadFactor(readFactor); + log.setWriteFactor(writeFactor); + pos += 8; + } + // oldFactor + boolean oldFactorExist = Bits.getBoolean(content, pos) == 1; + pos += 1; + if (oldFactorExist) { + int oldReadFactor = Bits.getInt(content, pos); + int oldWriteFactor = Bits.getInt(content, pos + 4); + log.setOldReadFactor(oldReadFactor); + log.setOldWriteFactor(oldWriteFactor); + pos += 8; } } -} \ No newline at end of file +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Encoder.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Encoder.java index 2df92a9bb..0b0013766 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Encoder.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v1/V1Encoder.java @@ -19,12 +19,14 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import com.alipay.sofa.jraft.entity.EnumOutter.EntryType; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.entity.codec.LogEntryEncoder; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import com.alipay.sofa.jraft.util.AsciiStringUtil; import com.alipay.sofa.jraft.util.Bits; @@ -51,7 +53,13 @@ public byte[] encode(final LogEntry log) { List peers = log.getPeers(); List oldPeers = log.getOldPeers(); ByteBuffer data = log.getData(); - + Boolean enableFlexible = log.getEnableFlexible(); + LogOutter.Quorum quorum = log.getQuorum(); + LogOutter.Quorum oldQuorum = log.getOldQuorum(); + final Integer readFactor = log.getReadFactor(); + final Integer writeFactor = log.getWriteFactor(); + final Integer oldReadFactor = log.getOldReadFactor(); + final Integer oldWriteFactor = log.getOldWriteFactor(); // magic number 1 byte int totalLen = 1; final int iType = type.getNumber(); @@ -88,8 +96,20 @@ public byte[] encode(final LogEntry log) { } } - final int bodyLen = data != null ? data.remaining() : 0; + final int bodyLen = data != null ? data.remaining() + 4 : 4; totalLen += bodyLen; + // isEnableFlexible + totalLen += 1; + // quorum + totalLen += quorum != null ? 8 + 1 : 1; + // oldQuorum + totalLen += oldQuorum != null ? 8 + 1 : 1; + // factor + totalLen += Objects.nonNull(readFactor) && Objects.nonNull(writeFactor) && readFactor != 0 && writeFactor != 0 ? 8 + 1 + : 1; + // oldFactor + totalLen += Objects.nonNull(oldReadFactor) && Objects.nonNull(oldWriteFactor) && oldReadFactor != 0 + && oldWriteFactor != 0 ? 8 + 1 : 1; final byte[] content = new byte[totalLen]; // {0} magic @@ -122,9 +142,83 @@ public byte[] encode(final LogEntry log) { } // data if (data != null) { - System.arraycopy(data.array(), data.position(), content, pos, data.remaining()); + int remaining = data.remaining(); + Bits.putInt(content, pos, remaining); + System.arraycopy(data.array(), data.position(), content, pos + 4, data.remaining()); + pos += data.remaining() + 4; + } else { + Bits.putInt(content, pos, 0); + pos += 4; } + // Flexible Raft V1 Format: + // isEnableFlexible quorumExist quorum oldQuorumExist oldQuorum factorExist readFactor+writeFactor oldFactorExist oldReadFactor+oldWriteFactor + // (Boolean)1 (Boolean)1 2*(int)4 (Boolean)1 2*(int)4 (Boolean)1 (int)8 (Boolean)1 (int)8 + + // isEnableFlexible + if (Objects.isNull(enableFlexible)) { + Bits.putBoolean(content, pos, 0); + } else { + Bits.putBoolean(content, pos, enableFlexible ? 1 : 0); + } + pos += 1; + // quorum + byte quorumExist; + if (Objects.nonNull(quorum)) { + quorumExist = 1; + int r = quorum.getR(); + int w = quorum.getW(); + Bits.putBoolean(content, pos, quorumExist); + Bits.putInt(content, pos + 1, r); + Bits.putInt(content, pos + 5, w); + pos += 9; + } else { + quorumExist = 0; + Bits.putBoolean(content, pos, quorumExist); + pos += 1; + } + // oldQuorum + byte oldQuorumExist; + if (Objects.nonNull(oldQuorum)) { + oldQuorumExist = 1; + int r = oldQuorum.getR(); + int w = oldQuorum.getW(); + Bits.putBoolean(content, pos, oldQuorumExist); + Bits.putInt(content, pos + 1, r); + Bits.putInt(content, pos + 5, w); + pos += 9; + } else { + oldQuorumExist = 0; + Bits.putBoolean(content, pos, oldQuorumExist); + pos += 1; + } + // readFactor and writeFactor + byte factorExist; + if (Objects.nonNull(readFactor) && Objects.nonNull(writeFactor) && readFactor != 0 && writeFactor != 0) { + factorExist = 1; + Bits.putBoolean(content, pos, factorExist); + Bits.putInt(content, pos + 1, readFactor); + Bits.putInt(content, pos + 5, writeFactor); + pos += 9; + } else { + factorExist = 0; + Bits.putBoolean(content, pos, factorExist); + pos += 1; + } + // oldReadFactor and oldWriteFactor + byte oldFactorExist; + if (Objects.nonNull(oldReadFactor) && Objects.nonNull(oldWriteFactor) && oldReadFactor != 0 + && oldWriteFactor != 0) { + oldFactorExist = 1; + Bits.putInt(content, pos, oldFactorExist); + Bits.putInt(content, pos + 1, oldReadFactor); + Bits.putInt(content, pos + 5, oldWriteFactor); + pos += 9; + } else { + oldFactorExist = 0; + Bits.putBoolean(content, pos, oldFactorExist); + pos += 1; + } return content; } -} \ No newline at end of file +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/LogOutter.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/LogOutter.java index 822c4570d..39397ef0e 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/LogOutter.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/LogOutter.java @@ -30,6 +30,582 @@ public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry r registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); } + public interface QuorumOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.Quorum) + com.google.protobuf.MessageOrBuilder { + + /** + * optional int32 w = 1; + */ + boolean hasW(); + + /** + * optional int32 w = 1; + */ + int getW(); + + /** + * optional int32 r = 2; + */ + boolean hasR(); + + /** + * optional int32 r = 2; + */ + int getR(); + } + + /** + * Protobuf type {@code jraft.Quorum} + */ + public static final class Quorum extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.Quorum) + QuorumOrBuilder { + private static final long serialVersionUID = 0L; + + // Use Quorum.newBuilder() to construct. + private Quorum(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Quorum() { + w_ = 0; + r_ = 0; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Quorum(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + w_ = input.readInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + r_ = input.readInt32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.internal_static_jraft_Quorum_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.internal_static_jraft_Quorum_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.class, + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder.class); + } + + private int bitField0_; + public static final int W_FIELD_NUMBER = 1; + private int w_; + + /** + * optional int32 w = 1; + */ + public boolean hasW() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * optional int32 w = 1; + */ + public int getW() { + return w_; + } + + public static final int R_FIELD_NUMBER = 2; + private int r_; + + /** + * optional int32 r = 2; + */ + public boolean hasR() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * optional int32 r = 2; + */ + public int getR() { + return r_; + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt32(1, w_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, r_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(1, w_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, r_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum)) { + return super.equals(obj); + } + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum other = (com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum) obj; + + boolean result = true; + result = result && (hasW() == other.hasW()); + if (hasW()) { + result = result && (getW() == other.getW()); + } + result = result && (hasR() == other.hasR()); + if (hasR()) { + result = result && (getR() == other.getR()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasW()) { + hash = (37 * hash) + W_FIELD_NUMBER; + hash = (53 * hash) + getW(); + } + if (hasR()) { + hash = (37 * hash) + R_FIELD_NUMBER; + hash = (53 * hash) + getR(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.Quorum} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.Quorum) + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.internal_static_jraft_Quorum_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.internal_static_jraft_Quorum_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.class, + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + public Builder clear() { + super.clear(); + w_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + r_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.internal_static_jraft_Quorum_descriptor; + } + + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getDefaultInstanceForType() { + return com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance(); + } + + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum build() { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum buildPartial() { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum result = new com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.w_ = w_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.r_ = r_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum) { + return mergeFrom((com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum other) { + if (other == com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) + return this; + if (other.hasW()) { + setW(other.getW()); + } + if (other.hasR()) { + setR(other.getR()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private int w_; + + /** + * optional int32 w = 1; + */ + public boolean hasW() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * optional int32 w = 1; + */ + public int getW() { + return w_; + } + + /** + * optional int32 w = 1; + */ + public Builder setW(int value) { + bitField0_ |= 0x00000001; + w_ = value; + onChanged(); + return this; + } + + /** + * optional int32 w = 1; + */ + public Builder clearW() { + bitField0_ = (bitField0_ & ~0x00000001); + w_ = 0; + onChanged(); + return this; + } + + private int r_; + + /** + * optional int32 r = 2; + */ + public boolean hasR() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * optional int32 r = 2; + */ + public int getR() { + return r_; + } + + /** + * optional int32 r = 2; + */ + public Builder setR(int value) { + bitField0_ |= 0x00000002; + r_ = value; + onChanged(); + return this; + } + + /** + * optional int32 r = 2; + */ + public Builder clearR() { + bitField0_ = (bitField0_ & ~0x00000002); + r_ = 0; + onChanged(); + return this; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.Quorum) + } + + // @@protoc_insertion_point(class_scope:jraft.Quorum) + private static final com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum(); + } + + public static com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public Quorum parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new Quorum(input, + extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + public interface PBLogEntryOrBuilder extends // @@protoc_insertion_point(interface_extends:jraft.PBLogEntry) com.google.protobuf.MessageOrBuilder { @@ -143,6 +719,86 @@ public interface PBLogEntryOrBuilder extends * repeated bytes old_learners = 9; */ com.google.protobuf.ByteString getOldLearners(int index); + + /** + * optional int32 read_factor = 10; + */ + boolean hasReadFactor(); + + /** + * optional int32 read_factor = 10; + */ + int getReadFactor(); + + /** + * optional int32 write_factor = 11; + */ + boolean hasWriteFactor(); + + /** + * optional int32 write_factor = 11; + */ + int getWriteFactor(); + + /** + * optional int32 old_read_factor = 12; + */ + boolean hasOldReadFactor(); + + /** + * optional int32 old_read_factor = 12; + */ + int getOldReadFactor(); + + /** + * optional int32 old_write_factor = 13; + */ + boolean hasOldWriteFactor(); + + /** + * optional int32 old_write_factor = 13; + */ + int getOldWriteFactor(); + + /** + * optional bool is_enable_flexible = 14; + */ + boolean hasIsEnableFlexible(); + + /** + * optional bool is_enable_flexible = 14; + */ + boolean getIsEnableFlexible(); + + /** + * optional .jraft.Quorum quorum = 15; + */ + boolean hasQuorum(); + + /** + * optional .jraft.Quorum quorum = 15; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum(); + + /** + * optional .jraft.Quorum quorum = 15; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder(); + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + boolean hasOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum(); + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder(); } /** @@ -168,6 +824,11 @@ private PBLogEntry() { checksum_ = 0L; learners_ = java.util.Collections.emptyList(); oldLearners_ = java.util.Collections.emptyList(); + readFactor_ = 0; + writeFactor_ = 0; + oldReadFactor_ = 0; + oldWriteFactor_ = 0; + isEnableFlexible_ = false; } @java.lang.Override @@ -263,6 +924,59 @@ private PBLogEntry(com.google.protobuf.CodedInputStream input, oldLearners_.add(input.readBytes()); break; } + case 80: { + bitField0_ |= 0x00000020; + readFactor_ = input.readInt32(); + break; + } + case 88: { + bitField0_ |= 0x00000040; + writeFactor_ = input.readInt32(); + break; + } + case 96: { + bitField0_ |= 0x00000080; + oldReadFactor_ = input.readInt32(); + break; + } + case 104: { + bitField0_ |= 0x00000100; + oldWriteFactor_ = input.readInt32(); + break; + } + case 112: { + bitField0_ |= 0x00000200; + isEnableFlexible_ = input.readBool(); + break; + } + case 122: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000400) == 0x00000400)) { + subBuilder = quorum_.toBuilder(); + } + quorum_ = input.readMessage(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(quorum_); + quorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000400; + break; + } + case 130: { + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder subBuilder = null; + if (((bitField0_ & 0x00000800) == 0x00000800)) { + subBuilder = oldQuorum_.toBuilder(); + } + oldQuorum_ = input.readMessage( + com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(oldQuorum_); + oldQuorum_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000800; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -481,6 +1195,143 @@ public com.google.protobuf.ByteString getOldLearners(int index) { return oldLearners_.get(index); } + public static final int READ_FACTOR_FIELD_NUMBER = 10; + private int readFactor_; + + /** + * optional int32 read_factor = 10; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + + /** + * optional int32 read_factor = 10; + */ + public int getReadFactor() { + return readFactor_; + } + + public static final int WRITE_FACTOR_FIELD_NUMBER = 11; + private int writeFactor_; + + /** + * optional int32 write_factor = 11; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + + /** + * optional int32 write_factor = 11; + */ + public int getWriteFactor() { + return writeFactor_; + } + + public static final int OLD_READ_FACTOR_FIELD_NUMBER = 12; + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 12; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + + /** + * optional int32 old_read_factor = 12; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + public static final int OLD_WRITE_FACTOR_FIELD_NUMBER = 13; + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 13; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + + /** + * optional int32 old_write_factor = 13; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + public static final int IS_ENABLE_FLEXIBLE_FIELD_NUMBER = 14; + private boolean isEnableFlexible_; + + /** + * optional bool is_enable_flexible = 14; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional bool is_enable_flexible = 14; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + public static final int QUORUM_FIELD_NUMBER = 15; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_; + + /** + * optional .jraft.Quorum quorum = 15; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : quorum_; + } + + public static final int OLD_QUORUM_FIELD_NUMBER = 16; + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_; + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance() + : oldQuorum_; + } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -538,6 +1389,27 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io for (int i = 0; i < oldLearners_.size(); i++) { output.writeBytes(9, oldLearners_.get(i)); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeInt32(10, readFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeInt32(11, writeFactor_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeInt32(12, oldReadFactor_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeInt32(13, oldWriteFactor_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBool(14, isEnableFlexible_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeMessage(15, getQuorum()); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeMessage(16, getOldQuorum()); + } unknownFields.writeTo(output); } @@ -594,6 +1466,27 @@ public int getSerializedSize() { size += dataSize; size += 1 * getOldLearnersList().size(); } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(10, readFactor_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(11, writeFactor_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(12, oldReadFactor_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(13, oldWriteFactor_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += com.google.protobuf.CodedOutputStream.computeBoolSize(14, isEnableFlexible_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(15, getQuorum()); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(16, getOldQuorum()); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -634,6 +1527,34 @@ public boolean equals(final java.lang.Object obj) { } result = result && getLearnersList().equals(other.getLearnersList()); result = result && getOldLearnersList().equals(other.getOldLearnersList()); + result = result && (hasReadFactor() == other.hasReadFactor()); + if (hasReadFactor()) { + result = result && (getReadFactor() == other.getReadFactor()); + } + result = result && (hasWriteFactor() == other.hasWriteFactor()); + if (hasWriteFactor()) { + result = result && (getWriteFactor() == other.getWriteFactor()); + } + result = result && (hasOldReadFactor() == other.hasOldReadFactor()); + if (hasOldReadFactor()) { + result = result && (getOldReadFactor() == other.getOldReadFactor()); + } + result = result && (hasOldWriteFactor() == other.hasOldWriteFactor()); + if (hasOldWriteFactor()) { + result = result && (getOldWriteFactor() == other.getOldWriteFactor()); + } + result = result && (hasIsEnableFlexible() == other.hasIsEnableFlexible()); + if (hasIsEnableFlexible()) { + result = result && (getIsEnableFlexible() == other.getIsEnableFlexible()); + } + result = result && (hasQuorum() == other.hasQuorum()); + if (hasQuorum()) { + result = result && getQuorum().equals(other.getQuorum()); + } + result = result && (hasOldQuorum() == other.hasOldQuorum()); + if (hasOldQuorum()) { + result = result && getOldQuorum().equals(other.getOldQuorum()); + } result = result && unknownFields.equals(other.unknownFields); return result; } @@ -677,9 +1598,37 @@ public int hashCode() { hash = (37 * hash) + LEARNERS_FIELD_NUMBER; hash = (53 * hash) + getLearnersList().hashCode(); } - if (getOldLearnersCount() > 0) { - hash = (37 * hash) + OLD_LEARNERS_FIELD_NUMBER; - hash = (53 * hash) + getOldLearnersList().hashCode(); + if (getOldLearnersCount() > 0) { + hash = (37 * hash) + OLD_LEARNERS_FIELD_NUMBER; + hash = (53 * hash) + getOldLearnersList().hashCode(); + } + if (hasReadFactor()) { + hash = (37 * hash) + READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getReadFactor(); + } + if (hasWriteFactor()) { + hash = (37 * hash) + WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getWriteFactor(); + } + if (hasOldReadFactor()) { + hash = (37 * hash) + OLD_READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldReadFactor(); + } + if (hasOldWriteFactor()) { + hash = (37 * hash) + OLD_WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getOldWriteFactor(); + } + if (hasIsEnableFlexible()) { + hash = (37 * hash) + IS_ENABLE_FLEXIBLE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getIsEnableFlexible()); + } + if (hasQuorum()) { + hash = (37 * hash) + QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getQuorum().hashCode(); + } + if (hasOldQuorum()) { + hash = (37 * hash) + OLD_QUORUM_FIELD_NUMBER; + hash = (53 * hash) + getOldQuorum().hashCode(); } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; @@ -804,6 +1753,8 @@ private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getQuorumFieldBuilder(); + getOldQuorumFieldBuilder(); } } @@ -827,6 +1778,28 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000080); oldLearners_ = java.util.Collections.emptyList(); bitField0_ = (bitField0_ & ~0x00000100); + readFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000200); + writeFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000400); + oldReadFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000800); + oldWriteFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00001000); + isEnableFlexible_ = false; + bitField0_ = (bitField0_ & ~0x00002000); + if (quorumBuilder_ == null) { + quorum_ = null; + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00004000); + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00008000); return this; } @@ -891,6 +1864,42 @@ public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.PBLogEntry buildPartial() bitField0_ = (bitField0_ & ~0x00000100); } result.oldLearners_ = oldLearners_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000020; + } + result.readFactor_ = readFactor_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000040; + } + result.writeFactor_ = writeFactor_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000080; + } + result.oldReadFactor_ = oldReadFactor_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000100; + } + result.oldWriteFactor_ = oldWriteFactor_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00000200; + } + result.isEnableFlexible_ = isEnableFlexible_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00000400; + } + if (quorumBuilder_ == null) { + result.quorum_ = quorum_; + } else { + result.quorum_ = quorumBuilder_.build(); + } + if (((from_bitField0_ & 0x00008000) == 0x00008000)) { + to_bitField0_ |= 0x00000800; + } + if (oldQuorumBuilder_ == null) { + result.oldQuorum_ = oldQuorum_; + } else { + result.oldQuorum_ = oldQuorumBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -989,6 +1998,27 @@ public Builder mergeFrom(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.PBLogEn } onChanged(); } + if (other.hasReadFactor()) { + setReadFactor(other.getReadFactor()); + } + if (other.hasWriteFactor()) { + setWriteFactor(other.getWriteFactor()); + } + if (other.hasOldReadFactor()) { + setOldReadFactor(other.getOldReadFactor()); + } + if (other.hasOldWriteFactor()) { + setOldWriteFactor(other.getOldWriteFactor()); + } + if (other.hasIsEnableFlexible()) { + setIsEnableFlexible(other.getIsEnableFlexible()); + } + if (other.hasQuorum()) { + mergeQuorum(other.getQuorum()); + } + if (other.hasOldQuorum()) { + mergeOldQuorum(other.getOldQuorum()); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -1522,6 +2552,426 @@ public Builder clearOldLearners() { return this; } + private int readFactor_; + + /** + * optional int32 read_factor = 10; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + + /** + * optional int32 read_factor = 10; + */ + public int getReadFactor() { + return readFactor_; + } + + /** + * optional int32 read_factor = 10; + */ + public Builder setReadFactor(int value) { + bitField0_ |= 0x00000200; + readFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 read_factor = 10; + */ + public Builder clearReadFactor() { + bitField0_ = (bitField0_ & ~0x00000200); + readFactor_ = 0; + onChanged(); + return this; + } + + private int writeFactor_; + + /** + * optional int32 write_factor = 11; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + + /** + * optional int32 write_factor = 11; + */ + public int getWriteFactor() { + return writeFactor_; + } + + /** + * optional int32 write_factor = 11; + */ + public Builder setWriteFactor(int value) { + bitField0_ |= 0x00000400; + writeFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 write_factor = 11; + */ + public Builder clearWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000400); + writeFactor_ = 0; + onChanged(); + return this; + } + + private int oldReadFactor_; + + /** + * optional int32 old_read_factor = 12; + */ + public boolean hasOldReadFactor() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + + /** + * optional int32 old_read_factor = 12; + */ + public int getOldReadFactor() { + return oldReadFactor_; + } + + /** + * optional int32 old_read_factor = 12; + */ + public Builder setOldReadFactor(int value) { + bitField0_ |= 0x00000800; + oldReadFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_read_factor = 12; + */ + public Builder clearOldReadFactor() { + bitField0_ = (bitField0_ & ~0x00000800); + oldReadFactor_ = 0; + onChanged(); + return this; + } + + private int oldWriteFactor_; + + /** + * optional int32 old_write_factor = 13; + */ + public boolean hasOldWriteFactor() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + + /** + * optional int32 old_write_factor = 13; + */ + public int getOldWriteFactor() { + return oldWriteFactor_; + } + + /** + * optional int32 old_write_factor = 13; + */ + public Builder setOldWriteFactor(int value) { + bitField0_ |= 0x00001000; + oldWriteFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 old_write_factor = 13; + */ + public Builder clearOldWriteFactor() { + bitField0_ = (bitField0_ & ~0x00001000); + oldWriteFactor_ = 0; + onChanged(); + return this; + } + + private boolean isEnableFlexible_; + + /** + * optional bool is_enable_flexible = 14; + */ + public boolean hasIsEnableFlexible() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + + /** + * optional bool is_enable_flexible = 14; + */ + public boolean getIsEnableFlexible() { + return isEnableFlexible_; + } + + /** + * optional bool is_enable_flexible = 14; + */ + public Builder setIsEnableFlexible(boolean value) { + bitField0_ |= 0x00002000; + isEnableFlexible_ = value; + onChanged(); + return this; + } + + /** + * optional bool is_enable_flexible = 14; + */ + public Builder clearIsEnableFlexible() { + bitField0_ = (bitField0_ & ~0x00002000); + isEnableFlexible_ = false; + onChanged(); + return this; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum quorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 quorumBuilder_; + + /** + * optional .jraft.Quorum quorum = 15; + */ + public boolean hasQuorum() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getQuorum() { + if (quorumBuilder_ == null) { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } else { + return quorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + quorum_ = value; + onChanged(); + } else { + quorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public Builder setQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (quorumBuilder_ == null) { + quorum_ = builderForValue.build(); + onChanged(); + } else { + quorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public Builder mergeQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (quorumBuilder_ == null) { + if (((bitField0_ & 0x00004000) == 0x00004000) && quorum_ != null + && quorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + quorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(quorum_) + .mergeFrom(value).buildPartial(); + } else { + quorum_ = value; + } + onChanged(); + } else { + quorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00004000; + return this; + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public Builder clearQuorum() { + if (quorumBuilder_ == null) { + quorum_ = null; + onChanged(); + } else { + quorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00004000); + return this; + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getQuorumBuilder() { + bitField0_ |= 0x00004000; + onChanged(); + return getQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getQuorumOrBuilder() { + if (quorumBuilder_ != null) { + return quorumBuilder_.getMessageOrBuilder(); + } else { + return quorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : quorum_; + } + } + + /** + * optional .jraft.Quorum quorum = 15; + */ + private com.google.protobuf.SingleFieldBuilderV3 getQuorumFieldBuilder() { + if (quorumBuilder_ == null) { + quorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getQuorum(), getParentForChildren(), isClean()); + quorum_ = null; + } + return quorumBuilder_; + } + + private com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum oldQuorum_ = null; + private com.google.protobuf.SingleFieldBuilderV3 oldQuorumBuilder_; + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public boolean hasOldQuorum() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum getOldQuorum() { + if (oldQuorumBuilder_ == null) { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } else { + return oldQuorumBuilder_.getMessage(); + } + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + oldQuorum_ = value; + onChanged(); + } else { + oldQuorumBuilder_.setMessage(value); + } + bitField0_ |= 0x00008000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public Builder setOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder builderForValue) { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = builderForValue.build(); + onChanged(); + } else { + oldQuorumBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00008000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public Builder mergeOldQuorum(com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum value) { + if (oldQuorumBuilder_ == null) { + if (((bitField0_ & 0x00008000) == 0x00008000) && oldQuorum_ != null + && oldQuorum_ != com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.getDefaultInstance()) { + oldQuorum_ = com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.newBuilder(oldQuorum_) + .mergeFrom(value).buildPartial(); + } else { + oldQuorum_ = value; + } + onChanged(); + } else { + oldQuorumBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00008000; + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public Builder clearOldQuorum() { + if (oldQuorumBuilder_ == null) { + oldQuorum_ = null; + onChanged(); + } else { + oldQuorumBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00008000); + return this; + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum.Builder getOldQuorumBuilder() { + bitField0_ |= 0x00008000; + onChanged(); + return getOldQuorumFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.QuorumOrBuilder getOldQuorumOrBuilder() { + if (oldQuorumBuilder_ != null) { + return oldQuorumBuilder_.getMessageOrBuilder(); + } else { + return oldQuorum_ == null ? com.alipay.sofa.jraft.entity.codec.v2.LogOutter.Quorum + .getDefaultInstance() : oldQuorum_; + } + } + + /** + * optional .jraft.Quorum old_quorum = 16; + */ + private com.google.protobuf.SingleFieldBuilderV3 getOldQuorumFieldBuilder() { + if (oldQuorumBuilder_ == null) { + oldQuorumBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getOldQuorum(), getParentForChildren(), isClean()); + oldQuorum_ = null; + } + return oldQuorumBuilder_; + } + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { return super.setUnknownFields(unknownFields); } @@ -1568,6 +3018,8 @@ public com.alipay.sofa.jraft.entity.codec.v2.LogOutter.PBLogEntry getDefaultInst } + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_Quorum_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_Quorum_fieldAccessorTable; private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_PBLogEntry_descriptor; private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_PBLogEntry_fieldAccessorTable; @@ -1577,13 +3029,18 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { private static com.google.protobuf.Descriptors.FileDescriptor descriptor; static { - java.lang.String[] descriptorData = { "\n\tlog.proto\022\005jraft\032\nenum.proto\"\263\001\n\nPBLog" - + "Entry\022\036\n\004type\030\001 \002(\0162\020.jraft.EntryType\022\014\n" - + "\004term\030\002 \002(\003\022\r\n\005index\030\003 \002(\003\022\r\n\005peers\030\004 \003(" - + "\014\022\021\n\told_peers\030\005 \003(\014\022\014\n\004data\030\006 \002(\014\022\020\n\010ch" - + "ecksum\030\007 \001(\003\022\020\n\010learners\030\010 \003(\014\022\024\n\014old_le" - + "arners\030\t \003(\014B2\n%com.alipay.sofa.jraft.en" - + "tity.codec.v2B\tLogOutter" }; + java.lang.String[] descriptorData = { "\n\tlog.proto\022\005jraft\032\nenum.proto\"\036\n\006Quorum" + + "\022\t\n\001w\030\001 \001(\005\022\t\n\001r\030\002 \001(\005\"\357\002\n\nPBLogEntry\022\036\n" + + "\004type\030\001 \002(\0162\020.jraft.EntryType\022\014\n\004term\030\002 " + + "\002(\003\022\r\n\005index\030\003 \002(\003\022\r\n\005peers\030\004 \003(\014\022\021\n\told" + + "_peers\030\005 \003(\014\022\014\n\004data\030\006 \002(\014\022\020\n\010checksum\030\007" + + " \001(\003\022\020\n\010learners\030\010 \003(\014\022\024\n\014old_learners\030\t" + + " \003(\014\022\023\n\013read_factor\030\n \001(\005\022\024\n\014write_facto" + + "r\030\013 \001(\005\022\027\n\017old_read_factor\030\014 \001(\005\022\030\n\020old_" + + "write_factor\030\r \001(\005\022\032\n\022is_enable_flexible" + + "\030\016 \001(\010\022\035\n\006quorum\030\017 \001(\0132\r.jraft.Quorum\022!\n" + + "\nold_quorum\030\020 \001(\0132\r.jraft.QuorumB2\n%com." + + "alipay.sofa.jraft.entity.codec.v2B\tLogOu" + "tter" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; @@ -1593,10 +3050,14 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protob com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] { com.alipay.sofa.jraft.entity.EnumOutter .getDescriptor(), }, assigner); - internal_static_jraft_PBLogEntry_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_jraft_Quorum_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_jraft_Quorum_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_Quorum_descriptor, new java.lang.String[] { "W", "R", }); + internal_static_jraft_PBLogEntry_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_jraft_PBLogEntry_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_jraft_PBLogEntry_descriptor, new java.lang.String[] { "Type", "Term", "Index", "Peers", - "OldPeers", "Data", "Checksum", "Learners", "OldLearners", }); + "OldPeers", "Data", "Checksum", "Learners", "OldLearners", "ReadFactor", "WriteFactor", "OldReadFactor", + "OldWriteFactor", "IsEnableFlexible", "Quorum", "OldQuorum", }); com.alipay.sofa.jraft.entity.EnumOutter.getDescriptor(); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Decoder.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Decoder.java index edc98c2b7..c07773f05 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Decoder.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Decoder.java @@ -104,6 +104,28 @@ public LogEntry decode(final byte[] bs) { log.setOldLearners(peers); } + if (entry.hasWriteFactor() || entry.hasReadFactor()) { + log.setWriteFactor(entry.getWriteFactor()); + log.setReadFactor(entry.getReadFactor()); + } + + if (entry.hasOldWriteFactor() || entry.hasOldReadFactor()) { + log.setOldWriteFactor(entry.getOldWriteFactor()); + log.setOldReadFactor(entry.getOldReadFactor()); + } + + if (entry.hasIsEnableFlexible()) { + log.setEnableFlexible(entry.getIsEnableFlexible()); + } + + if (entry.hasQuorum()) { + log.setQuorum(entry.getQuorum()); + } + + if (entry.hasOldQuorum()) { + log.setOldQuorum(entry.getOldQuorum()); + } + final ByteString data = entry.getData(); if (!data.isEmpty()) { log.setData(ByteBuffer.wrap(ZeroByteStringHelper.getByteArray(data))); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Encoder.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Encoder.java index 22605a9b2..3de343f5e 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Encoder.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/entity/codec/v2/V2Encoder.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.Objects; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; @@ -108,6 +109,28 @@ public byte[] encode(final LogEntry log) { builder.setData(log.getData() != null ? ZeroByteStringHelper.wrap(log.getData()) : ByteString.EMPTY); + if (log.haveFactorValue()) { + builder.setReadFactor(log.getReadFactor()); + builder.setWriteFactor(log.getWriteFactor()); + } + + if (log.haveOldFactorValue()) { + builder.setOldReadFactor(log.getOldReadFactor()); + builder.setOldWriteFactor(log.getOldWriteFactor()); + } + + if (Objects.nonNull(log.getEnableFlexible())) { + builder.setIsEnableFlexible(log.getEnableFlexible()); + } + + if (Objects.nonNull(log.getQuorum())) { + builder.setQuorum(log.getQuorum()); + } + + if (Objects.nonNull(log.getOldQuorum())) { + builder.setOldQuorum(log.getOldQuorum()); + } + final PBLogEntry pbLogEntry = builder.build(); final int bodyLen = pbLogEntry.getSerializedSize(); final byte[] ret = new byte[LogEntryV2CodecFactory.HEADER_SIZE + bodyLen]; diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/option/NodeOptions.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/option/NodeOptions.java index 81ef572dc..cd0e00dc8 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/option/NodeOptions.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/option/NodeOptions.java @@ -21,6 +21,7 @@ import com.alipay.sofa.jraft.StateMachine; import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.core.ElectionPriority; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.storage.SnapshotThrottle; import com.alipay.sofa.jraft.util.Copiable; import com.alipay.sofa.jraft.util.JRaftServiceLoader; @@ -425,6 +426,17 @@ public void setSharedSnapshotTimer(final boolean sharedSnapshotTimer) { this.sharedSnapshotTimer = sharedSnapshotTimer; } + public void setFactor(Integer readFactor, Integer writeFactor) { + this.initialConf.setReadFactor(readFactor); + this.initialConf.setWriteFactor(writeFactor); + this.initialConf.setQuorum(BallotFactory.buildFlexibleQuorum(readFactor, writeFactor, initialConf.getPeers() + .size())); + } + + public void enableFlexibleRaft(boolean enabled) { + this.initialConf.setEnableFlexible(enabled); + } + @Override public NodeOptions copy() { final NodeOptions nodeOptions = new NodeOptions(); @@ -453,7 +465,6 @@ public NodeOptions copy() { nodeOptions.setRpcProcessorThreadPoolSize(super.getRpcProcessorThreadPoolSize()); nodeOptions.setEnableRpcChecksum(super.isEnableRpcChecksum()); nodeOptions.setMetricRegistry(super.getMetricRegistry()); - return nodeOptions; } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliClientService.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliClientService.java index aa2871c3d..182b4e626 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliClientService.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliClientService.java @@ -30,6 +30,9 @@ */ public interface CliClientService extends ClientService { + Future resetFactor(Endpoint endpoint, CliRequests.ResetFactorRequest request, + RpcResponseClosure done); + /** * Adds a peer. * diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliRequests.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliRequests.java index 46b1fc4b9..bdd7d4fed 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliRequests.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/CliRequests.java @@ -15633,6 +15633,1721 @@ public com.alipay.sofa.jraft.rpc.CliRequests.LearnersOpResponse getDefaultInstan } + public interface ResetFactorRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.ResetFactorRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * required string group_id = 1; + */ + boolean hasGroupId(); + + /** + * required string group_id = 1; + */ + java.lang.String getGroupId(); + + /** + * required string group_id = 1; + */ + com.google.protobuf.ByteString getGroupIdBytes(); + + /** + * required string leader_id = 2; + */ + boolean hasLeaderId(); + + /** + * required string leader_id = 2; + */ + java.lang.String getLeaderId(); + + /** + * required string leader_id = 2; + */ + com.google.protobuf.ByteString getLeaderIdBytes(); + + /** + * optional int32 read_factor = 3; + */ + boolean hasReadFactor(); + + /** + * optional int32 read_factor = 3; + */ + int getReadFactor(); + + /** + * optional int32 write_factor = 4; + */ + boolean hasWriteFactor(); + + /** + * optional int32 write_factor = 4; + */ + int getWriteFactor(); + } + + /** + * Protobuf type {@code jraft.ResetFactorRequest} + */ + public static final class ResetFactorRequest extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.ResetFactorRequest) + ResetFactorRequestOrBuilder { + private static final long serialVersionUID = 0L; + + // Use ResetFactorRequest.newBuilder() to construct. + private ResetFactorRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ResetFactorRequest() { + groupId_ = ""; + leaderId_ = ""; + readFactor_ = 0; + writeFactor_ = 0; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ResetFactorRequest(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + groupId_ = bs; + break; + } + case 18: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000002; + leaderId_ = bs; + break; + } + case 24: { + bitField0_ |= 0x00000004; + readFactor_ = input.readInt32(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + writeFactor_ = input.readInt32(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.class, + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.Builder.class); + } + + private int bitField0_; + public static final int GROUP_ID_FIELD_NUMBER = 1; + private volatile java.lang.Object groupId_; + + /** + * required string group_id = 1; + */ + public boolean hasGroupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required string group_id = 1; + */ + public java.lang.String getGroupId() { + java.lang.Object ref = groupId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + groupId_ = s; + } + return s; + } + } + + /** + * required string group_id = 1; + */ + public com.google.protobuf.ByteString getGroupIdBytes() { + java.lang.Object ref = groupId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + groupId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int LEADER_ID_FIELD_NUMBER = 2; + private volatile java.lang.Object leaderId_; + + /** + * required string leader_id = 2; + */ + public boolean hasLeaderId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required string leader_id = 2; + */ + public java.lang.String getLeaderId() { + java.lang.Object ref = leaderId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + leaderId_ = s; + } + return s; + } + } + + /** + * required string leader_id = 2; + */ + public com.google.protobuf.ByteString getLeaderIdBytes() { + java.lang.Object ref = leaderId_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + leaderId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int READ_FACTOR_FIELD_NUMBER = 3; + private int readFactor_; + + /** + * optional int32 read_factor = 3; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional int32 read_factor = 3; + */ + public int getReadFactor() { + return readFactor_; + } + + public static final int WRITE_FACTOR_FIELD_NUMBER = 4; + private int writeFactor_; + + /** + * optional int32 write_factor = 4; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional int32 write_factor = 4; + */ + public int getWriteFactor() { + return writeFactor_; + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + if (!hasGroupId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasLeaderId()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 1, groupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 2, leaderId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, readFactor_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, writeFactor_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(1, groupId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(2, leaderId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(3, readFactor_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(4, writeFactor_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest)) { + return super.equals(obj); + } + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest other = (com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest) obj; + + boolean result = true; + result = result && (hasGroupId() == other.hasGroupId()); + if (hasGroupId()) { + result = result && getGroupId().equals(other.getGroupId()); + } + result = result && (hasLeaderId() == other.hasLeaderId()); + if (hasLeaderId()) { + result = result && getLeaderId().equals(other.getLeaderId()); + } + result = result && (hasReadFactor() == other.hasReadFactor()); + if (hasReadFactor()) { + result = result && (getReadFactor() == other.getReadFactor()); + } + result = result && (hasWriteFactor() == other.hasWriteFactor()); + if (hasWriteFactor()) { + result = result && (getWriteFactor() == other.getWriteFactor()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasGroupId()) { + hash = (37 * hash) + GROUP_ID_FIELD_NUMBER; + hash = (53 * hash) + getGroupId().hashCode(); + } + if (hasLeaderId()) { + hash = (37 * hash) + LEADER_ID_FIELD_NUMBER; + hash = (53 * hash) + getLeaderId().hashCode(); + } + if (hasReadFactor()) { + hash = (37 * hash) + READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getReadFactor(); + } + if (hasWriteFactor()) { + hash = (37 * hash) + WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getWriteFactor(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.ResetFactorRequest} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.ResetFactorRequest) + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.class, + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + public Builder clear() { + super.clear(); + groupId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + leaderId_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + readFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + writeFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorRequest_descriptor; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest getDefaultInstanceForType() { + return com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.getDefaultInstance(); + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest build() { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest buildPartial() { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest result = new com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.groupId_ = groupId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.leaderId_ = leaderId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.readFactor_ = readFactor_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.writeFactor_ = writeFactor_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest) { + return mergeFrom((com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest other) { + if (other == com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest.getDefaultInstance()) + return this; + if (other.hasGroupId()) { + bitField0_ |= 0x00000001; + groupId_ = other.groupId_; + onChanged(); + } + if (other.hasLeaderId()) { + bitField0_ |= 0x00000002; + leaderId_ = other.leaderId_; + onChanged(); + } + if (other.hasReadFactor()) { + setReadFactor(other.getReadFactor()); + } + if (other.hasWriteFactor()) { + setWriteFactor(other.getWriteFactor()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasGroupId()) { + return false; + } + if (!hasLeaderId()) { + return false; + } + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object groupId_ = ""; + + /** + * required string group_id = 1; + */ + public boolean hasGroupId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required string group_id = 1; + */ + public java.lang.String getGroupId() { + java.lang.Object ref = groupId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + groupId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * required string group_id = 1; + */ + public com.google.protobuf.ByteString getGroupIdBytes() { + java.lang.Object ref = groupId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString + .copyFromUtf8((java.lang.String) ref); + groupId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * required string group_id = 1; + */ + public Builder setGroupId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + groupId_ = value; + onChanged(); + return this; + } + + /** + * required string group_id = 1; + */ + public Builder clearGroupId() { + bitField0_ = (bitField0_ & ~0x00000001); + groupId_ = getDefaultInstance().getGroupId(); + onChanged(); + return this; + } + + /** + * required string group_id = 1; + */ + public Builder setGroupIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + groupId_ = value; + onChanged(); + return this; + } + + private java.lang.Object leaderId_ = ""; + + /** + * required string leader_id = 2; + */ + public boolean hasLeaderId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required string leader_id = 2; + */ + public java.lang.String getLeaderId() { + java.lang.Object ref = leaderId_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + leaderId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * required string leader_id = 2; + */ + public com.google.protobuf.ByteString getLeaderIdBytes() { + java.lang.Object ref = leaderId_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString + .copyFromUtf8((java.lang.String) ref); + leaderId_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * required string leader_id = 2; + */ + public Builder setLeaderId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + leaderId_ = value; + onChanged(); + return this; + } + + /** + * required string leader_id = 2; + */ + public Builder clearLeaderId() { + bitField0_ = (bitField0_ & ~0x00000002); + leaderId_ = getDefaultInstance().getLeaderId(); + onChanged(); + return this; + } + + /** + * required string leader_id = 2; + */ + public Builder setLeaderIdBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + leaderId_ = value; + onChanged(); + return this; + } + + private int readFactor_; + + /** + * optional int32 read_factor = 3; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional int32 read_factor = 3; + */ + public int getReadFactor() { + return readFactor_; + } + + /** + * optional int32 read_factor = 3; + */ + public Builder setReadFactor(int value) { + bitField0_ |= 0x00000004; + readFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 read_factor = 3; + */ + public Builder clearReadFactor() { + bitField0_ = (bitField0_ & ~0x00000004); + readFactor_ = 0; + onChanged(); + return this; + } + + private int writeFactor_; + + /** + * optional int32 write_factor = 4; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional int32 write_factor = 4; + */ + public int getWriteFactor() { + return writeFactor_; + } + + /** + * optional int32 write_factor = 4; + */ + public Builder setWriteFactor(int value) { + bitField0_ |= 0x00000008; + writeFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 write_factor = 4; + */ + public Builder clearWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000008); + writeFactor_ = 0; + onChanged(); + return this; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.ResetFactorRequest) + } + + // @@protoc_insertion_point(class_scope:jraft.ResetFactorRequest) + private static final com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest(); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public ResetFactorRequest parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ResetFactorRequest( + input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface ResetFactorResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.ResetFactorResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * optional int32 read_factor = 1; + */ + boolean hasReadFactor(); + + /** + * optional int32 read_factor = 1; + */ + int getReadFactor(); + + /** + * optional int32 write_factor = 2; + */ + boolean hasWriteFactor(); + + /** + * optional int32 write_factor = 2; + */ + int getWriteFactor(); + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + boolean hasErrorResponse(); + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse getErrorResponse(); + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponseOrBuilder getErrorResponseOrBuilder(); + } + + /** + * Protobuf type {@code jraft.ResetFactorResponse} + */ + public static final class ResetFactorResponse extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.ResetFactorResponse) + ResetFactorResponseOrBuilder { + private static final long serialVersionUID = 0L; + + // Use ResetFactorResponse.newBuilder() to construct. + private ResetFactorResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ResetFactorResponse() { + readFactor_ = 0; + writeFactor_ = 0; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ResetFactorResponse(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + readFactor_ = input.readInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + writeFactor_ = input.readInt32(); + break; + } + case 794: { + com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + subBuilder = errorResponse_.toBuilder(); + } + errorResponse_ = input.readMessage( + com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(errorResponse_); + errorResponse_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.class, + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.Builder.class); + } + + private int bitField0_; + public static final int READ_FACTOR_FIELD_NUMBER = 1; + private int readFactor_; + + /** + * optional int32 read_factor = 1; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * optional int32 read_factor = 1; + */ + public int getReadFactor() { + return readFactor_; + } + + public static final int WRITE_FACTOR_FIELD_NUMBER = 2; + private int writeFactor_; + + /** + * optional int32 write_factor = 2; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * optional int32 write_factor = 2; + */ + public int getWriteFactor() { + return writeFactor_; + } + + public static final int ERRORRESPONSE_FIELD_NUMBER = 99; + private com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse errorResponse_; + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public boolean hasErrorResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse getErrorResponse() { + return errorResponse_ == null ? com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.getDefaultInstance() + : errorResponse_; + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponseOrBuilder getErrorResponseOrBuilder() { + return errorResponse_ == null ? com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.getDefaultInstance() + : errorResponse_; + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + if (hasErrorResponse()) { + if (!getErrorResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt32(1, readFactor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, writeFactor_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(99, getErrorResponse()); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(1, readFactor_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream.computeInt32Size(2, writeFactor_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream.computeMessageSize(99, getErrorResponse()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse)) { + return super.equals(obj); + } + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse other = (com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse) obj; + + boolean result = true; + result = result && (hasReadFactor() == other.hasReadFactor()); + if (hasReadFactor()) { + result = result && (getReadFactor() == other.getReadFactor()); + } + result = result && (hasWriteFactor() == other.hasWriteFactor()); + if (hasWriteFactor()) { + result = result && (getWriteFactor() == other.getWriteFactor()); + } + result = result && (hasErrorResponse() == other.hasErrorResponse()); + if (hasErrorResponse()) { + result = result && getErrorResponse().equals(other.getErrorResponse()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasReadFactor()) { + hash = (37 * hash) + READ_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getReadFactor(); + } + if (hasWriteFactor()) { + hash = (37 * hash) + WRITE_FACTOR_FIELD_NUMBER; + hash = (53 * hash) + getWriteFactor(); + } + if (hasErrorResponse()) { + hash = (37 * hash) + ERRORRESPONSE_FIELD_NUMBER; + hash = (53 * hash) + getErrorResponse().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.ResetFactorResponse} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.ResetFactorResponse) + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.class, + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + getErrorResponseFieldBuilder(); + } + } + + public Builder clear() { + super.clear(); + readFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + writeFactor_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + if (errorResponseBuilder_ == null) { + errorResponse_ = null; + } else { + errorResponseBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.rpc.CliRequests.internal_static_jraft_ResetFactorResponse_descriptor; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse getDefaultInstanceForType() { + return com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.getDefaultInstance(); + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse build() { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse buildPartial() { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse result = new com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.readFactor_ = readFactor_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.writeFactor_ = writeFactor_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (errorResponseBuilder_ == null) { + result.errorResponse_ = errorResponse_; + } else { + result.errorResponse_ = errorResponseBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse) { + return mergeFrom((com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse other) { + if (other == com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse.getDefaultInstance()) + return this; + if (other.hasReadFactor()) { + setReadFactor(other.getReadFactor()); + } + if (other.hasWriteFactor()) { + setWriteFactor(other.getWriteFactor()); + } + if (other.hasErrorResponse()) { + mergeErrorResponse(other.getErrorResponse()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (hasErrorResponse()) { + if (!getErrorResponse().isInitialized()) { + return false; + } + } + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse) e + .getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private int readFactor_; + + /** + * optional int32 read_factor = 1; + */ + public boolean hasReadFactor() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * optional int32 read_factor = 1; + */ + public int getReadFactor() { + return readFactor_; + } + + /** + * optional int32 read_factor = 1; + */ + public Builder setReadFactor(int value) { + bitField0_ |= 0x00000001; + readFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 read_factor = 1; + */ + public Builder clearReadFactor() { + bitField0_ = (bitField0_ & ~0x00000001); + readFactor_ = 0; + onChanged(); + return this; + } + + private int writeFactor_; + + /** + * optional int32 write_factor = 2; + */ + public boolean hasWriteFactor() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * optional int32 write_factor = 2; + */ + public int getWriteFactor() { + return writeFactor_; + } + + /** + * optional int32 write_factor = 2; + */ + public Builder setWriteFactor(int value) { + bitField0_ |= 0x00000002; + writeFactor_ = value; + onChanged(); + return this; + } + + /** + * optional int32 write_factor = 2; + */ + public Builder clearWriteFactor() { + bitField0_ = (bitField0_ & ~0x00000002); + writeFactor_ = 0; + onChanged(); + return this; + } + + private com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse errorResponse_ = null; + private com.google.protobuf.SingleFieldBuilderV3 errorResponseBuilder_; + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public boolean hasErrorResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse getErrorResponse() { + if (errorResponseBuilder_ == null) { + return errorResponse_ == null ? com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse + .getDefaultInstance() : errorResponse_; + } else { + return errorResponseBuilder_.getMessage(); + } + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public Builder setErrorResponse(com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse value) { + if (errorResponseBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + errorResponse_ = value; + onChanged(); + } else { + errorResponseBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public Builder setErrorResponse(com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.Builder builderForValue) { + if (errorResponseBuilder_ == null) { + errorResponse_ = builderForValue.build(); + onChanged(); + } else { + errorResponseBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public Builder mergeErrorResponse(com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse value) { + if (errorResponseBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && errorResponse_ != null + && errorResponse_ != com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.getDefaultInstance()) { + errorResponse_ = com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.newBuilder(errorResponse_) + .mergeFrom(value).buildPartial(); + } else { + errorResponse_ = value; + } + onChanged(); + } else { + errorResponseBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public Builder clearErrorResponse() { + if (errorResponseBuilder_ == null) { + errorResponse_ = null; + onChanged(); + } else { + errorResponseBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse.Builder getErrorResponseBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getErrorResponseFieldBuilder().getBuilder(); + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + public com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponseOrBuilder getErrorResponseOrBuilder() { + if (errorResponseBuilder_ != null) { + return errorResponseBuilder_.getMessageOrBuilder(); + } else { + return errorResponse_ == null ? com.alipay.sofa.jraft.rpc.RpcRequests.ErrorResponse + .getDefaultInstance() : errorResponse_; + } + } + + /** + * optional .jraft.ErrorResponse errorResponse = 99; + */ + private com.google.protobuf.SingleFieldBuilderV3 getErrorResponseFieldBuilder() { + if (errorResponseBuilder_ == null) { + errorResponseBuilder_ = new com.google.protobuf.SingleFieldBuilderV3( + getErrorResponse(), getParentForChildren(), isClean()); + errorResponse_ = null; + } + return errorResponseBuilder_; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.ResetFactorResponse) + } + + // @@protoc_insertion_point(class_scope:jraft.ResetFactorResponse) + private static final com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse(); + } + + public static com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public ResetFactorResponse parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new ResetFactorResponse( + input, extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_AddPeerRequest_descriptor; private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_AddPeerRequest_fieldAccessorTable; private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_AddPeerResponse_descriptor; @@ -15667,6 +17382,10 @@ public com.alipay.sofa.jraft.rpc.CliRequests.LearnersOpResponse getDefaultInstan private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_ResetLearnersRequest_fieldAccessorTable; private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_LearnersOpResponse_descriptor; private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_LearnersOpResponse_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_ResetFactorRequest_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_ResetFactorRequest_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_ResetFactorResponse_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_ResetFactorResponse_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { return descriptor; @@ -15710,8 +17429,13 @@ public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + "der_id\030\002 \002(\t\022\020\n\010learners\030\003 \003(\t\"m\n\022Learne" + "rsOpResponse\022\024\n\014old_learners\030\001 \003(\t\022\024\n\014ne" + "w_learners\030\002 \003(\t\022+\n\rerrorResponse\030c \001(\0132" - + "\024.jraft.ErrorResponseB(\n\031com.alipay.sofa" - + ".jraft.rpcB\013CliRequests" }; + + "\024.jraft.ErrorResponse\"d\n\022ResetFactorRequ" + + "est\022\020\n\010group_id\030\001 \002(\t\022\021\n\tleader_id\030\002 \002(\t" + + "\022\023\n\013read_factor\030\003 \001(\005\022\024\n\014write_factor\030\004 " + + "\001(\005\"m\n\023ResetFactorResponse\022\023\n\013read_facto" + + "r\030\001 \001(\005\022\024\n\014write_factor\030\002 \001(\005\022+\n\rerrorRe" + + "sponse\030c \001(\0132\024.jraft.ErrorResponseB(\n\031co" + + "m.alipay.sofa.jraft.rpcB\013CliRequests" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protobuf.Descriptors.FileDescriptor root) { descriptor = root; @@ -15786,6 +17510,14 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protob internal_static_jraft_LearnersOpResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( internal_static_jraft_LearnersOpResponse_descriptor, new java.lang.String[] { "OldLearners", "NewLearners", "ErrorResponse", }); + internal_static_jraft_ResetFactorRequest_descriptor = getDescriptor().getMessageTypes().get(17); + internal_static_jraft_ResetFactorRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_ResetFactorRequest_descriptor, new java.lang.String[] { "GroupId", "LeaderId", + "ReadFactor", "WriteFactor", }); + internal_static_jraft_ResetFactorResponse_descriptor = getDescriptor().getMessageTypes().get(18); + internal_static_jraft_ResetFactorResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_ResetFactorResponse_descriptor, new java.lang.String[] { "ReadFactor", "WriteFactor", + "ErrorResponse", }); com.alipay.sofa.jraft.rpc.RpcRequests.getDescriptor(); } diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java index 38dc6b614..17743f20d 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/RaftRpcServerFactory.java @@ -26,6 +26,7 @@ import com.alipay.sofa.jraft.rpc.impl.cli.GetPeersRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.RemoveLearnersRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.RemovePeerRequestProcessor; +import com.alipay.sofa.jraft.rpc.impl.cli.ResetFactorRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.ResetLearnersRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.ResetPeerRequestProcessor; import com.alipay.sofa.jraft.rpc.impl.cli.SnapshotRequestProcessor; @@ -117,6 +118,7 @@ public static void addRaftRequestProcessors(final RpcServer rpcServer, final Exe rpcServer.registerProcessor(new AddLearnersRequestProcessor(cliExecutor)); rpcServer.registerProcessor(new RemoveLearnersRequestProcessor(cliExecutor)); rpcServer.registerProcessor(new ResetLearnersRequestProcessor(cliExecutor)); + rpcServer.registerProcessor(new ResetFactorRequestProcessor(cliExecutor)); } /** diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java index 2e245b664..781e37696 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ChangePeersRequestProcessor.java @@ -20,6 +20,7 @@ import java.util.concurrent.Executor; import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.error.RaftError; import com.alipay.sofa.jraft.rpc.CliRequests.ChangePeersRequest; @@ -65,6 +66,7 @@ protected Message processRequest0(final CliRequestContext ctx, final ChangePeers .newResponse(defaultResp(), RaftError.EINVAL, "Fail to parse peer id %s", peerIdStr); } } + conf.setQuorum(BallotFactory.buildMajorityQuorum(conf.size())); LOG.info("Receive ChangePeersRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), conf); ctx.node.changePeers(conf, status -> { diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java index 679c96301..7381dfcd2 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/CliClientServiceImpl.java @@ -52,6 +52,12 @@ public class CliClientServiceImpl extends AbstractClientService implements CliCl private CliOptions cliOptions; + @Override + public Future resetFactor(final Endpoint endpoint, final CliRequests.ResetFactorRequest request, + final RpcResponseClosure done) { + return invokeWithDone(endpoint, request, done, this.cliOptions.getTimeoutMs()); + } + @Override public synchronized boolean init(final RpcOptions rpcOptions) { boolean ret = super.init(rpcOptions); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetFactorRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetFactorRequestProcessor.java new file mode 100644 index 000000000..e905bbb64 --- /dev/null +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetFactorRequestProcessor.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.rpc.impl.cli; + +import com.alipay.sofa.jraft.Node; +import com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorResponse; +import com.alipay.sofa.jraft.rpc.CliRequests.ResetFactorRequest; +import com.alipay.sofa.jraft.rpc.RpcRequestClosure; +import com.google.protobuf.Message; + +import java.util.concurrent.Executor; + +/** + * Reset factor request processor + * + * @author Akai + */ +public class ResetFactorRequestProcessor extends BaseCliRequestProcessor { + + public ResetFactorRequestProcessor(Executor executor) { + super(executor, ResetFactorResponse.getDefaultInstance()); + } + + @Override + public String interest() { + return ResetFactorRequest.class.getName(); + } + + @Override + protected String getPeerId(ResetFactorRequest request) { + return request.getLeaderId(); + } + + @Override + protected String getGroupId(ResetFactorRequest request) { + return request.getGroupId(); + } + + @Override + protected Message processRequest0(CliRequestContext ctx, ResetFactorRequest request, RpcRequestClosure done) { + Node node = ctx.node; + int readFactor = request.getReadFactor(); + int writeFactor = request.getWriteFactor(); + LOG.info("Receive AddPeerRequest to {} from {}, change readFactor to {} , writeFactor to {}", + ctx.node.getNodeId(), done.getRpcCtx().getRemoteAddress(), readFactor, writeFactor); + node.resetFactor(readFactor, writeFactor, status -> { + if (!status.isOk()) { + done.run(status); + } else { + final ResetFactorResponse.Builder rb = ResetFactorResponse.newBuilder(); + rb.setReadFactor(readFactor); + rb.setWriteFactor(writeFactor); + done.sendResponse(rb.build()); + } + }); + return null; + } +} diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java index 26bbee11f..3f3d9e2c2 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/rpc/impl/cli/ResetPeerRequestProcessor.java @@ -20,6 +20,7 @@ import com.alipay.sofa.jraft.Status; import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.error.RaftError; import com.alipay.sofa.jraft.rpc.CliRequests.ResetPeerRequest; @@ -64,6 +65,7 @@ protected Message processRequest0(final CliRequestContext ctx, final ResetPeerRe .newResponse(defaultResp(), RaftError.EINVAL, "Fail to parse peer id %s", peerIdStr); } } + newConf.setQuorum(BallotFactory.buildMajorityQuorum(request.getNewPeersList().size())); LOG.info("Receive ResetPeerRequest to {} from {}, new conf is {}", ctx.node.getNodeId(), done.getRpcCtx() .getRemoteAddress(), newConf); final Status st = ctx.node.resetPeers(newConf); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/LogManagerImpl.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/LogManagerImpl.java index 4f08e89f8..e0ce91e36 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/LogManagerImpl.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/LogManagerImpl.java @@ -17,6 +17,7 @@ package com.alipay.sofa.jraft.storage.impl; import java.util.ArrayList; +import java.util.Objects; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -27,6 +28,10 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.entity.LogEntry; +import com.alipay.sofa.jraft.entity.LogId; +import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.util.ThreadPoolsFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,9 +44,6 @@ import com.alipay.sofa.jraft.core.NodeMetrics; import com.alipay.sofa.jraft.entity.EnumOutter.EntryType; import com.alipay.sofa.jraft.entity.EnumOutter.ErrorType; -import com.alipay.sofa.jraft.entity.LogEntry; -import com.alipay.sofa.jraft.entity.LogId; -import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.entity.RaftOutter.SnapshotMeta; import com.alipay.sofa.jraft.error.LogEntryCorruptedException; import com.alipay.sofa.jraft.error.RaftError; @@ -198,7 +200,7 @@ public boolean init(final LogManagerOptions opts) { * Use timeout strategy in log manager. If timeout happens, it will called reportError to halt the node. */ .setWaitStrategy(new TimeoutBlockingWaitStrategy( - this.raftOptions.getDisruptorPublishEventWaitTimeoutSecs(), TimeUnit.SECONDS)) // + this.raftOptions.getDisruptorPublishEventWaitTimeoutSecs(), TimeUnit.SECONDS)) // .build(); this.disruptor.handleEventsWith(new StableClosureEventHandler()); this.disruptor.setDefaultExceptionHandler(new LogExceptionHandler(this.getClass().getSimpleName(), @@ -206,7 +208,7 @@ public boolean init(final LogManagerOptions opts) { this.diskQueue = this.disruptor.start(); if (this.nodeMetrics.getMetricRegistry() != null) { this.nodeMetrics.getMetricRegistry().register("jraft-log-manager-disruptor", - new DisruptorMetricSet(this.diskQueue)); + new DisruptorMetricSet(this.diskQueue)); } } finally { this.writeLock.unlock(); @@ -295,7 +297,7 @@ void await() throws InterruptedException { @Override public void appendEntries(final List entries, final StableClosure done) { - assert(done != null); + assert (done != null); Requires.requireNonNull(done, "done"); if (this.hasError) { @@ -318,12 +320,19 @@ public void appendEntries(final List entries, final StableClosure done entry.setChecksum(entry.checksum()); } if (entry.getType() == EntryType.ENTRY_TYPE_CONFIGURATION) { + Quorum quorum = new Quorum(entry.getQuorum().getW(), entry.getQuorum().getR()); + Configuration newConf = new Configuration(entry.getPeers(), entry.getLearners(), quorum, + entry.getReadFactor(), entry.getWriteFactor(), entry.getEnableFlexible()); Configuration oldConf = new Configuration(); if (entry.getOldPeers() != null) { - oldConf = new Configuration(entry.getOldPeers(), entry.getOldLearners()); + Quorum oldQuorum = null; + if(Objects.nonNull(entry.getOldQuorum())){ + oldQuorum = new Quorum(entry.getOldQuorum().getW(), entry.getOldQuorum().getR()); + } + oldConf = new Configuration(entry.getOldPeers(), entry.getOldLearners(), oldQuorum, entry.getOldReadFactor(), entry.getOldWriteFactor(), entry.getEnableFlexible()); } final ConfigurationEntry conf = new ConfigurationEntry(entry.getId(), - new Configuration(entry.getPeers(), entry.getLearners()), oldConf); + newConf, oldConf); this.configManager.add(conf); } } @@ -357,13 +366,13 @@ public void appendEntries(final List entries, final StableClosure done * @param type */ private void offerEvent(final StableClosure done, final EventType type) { - assert(done != null); + assert (done != null); if (this.stopped) { ThreadPoolsFactory.runClosureInThread(this.groupId, done, new Status(RaftError.ESTOP, "Log manager is stopped.")); return; } - this.diskQueue.publishEvent((event, sequence) -> { + this.diskQueue.publishEvent((event, sequence) -> { event.reset(); event.type = type; event.done = done; @@ -606,7 +615,6 @@ public void setSnapshot(final SnapshotMeta meta) { } final Configuration conf = confFromMeta(meta); final Configuration oldConf = oldConfFromMeta(meta); - final ConfigurationEntry entry = new ConfigurationEntry(new LogId(meta.getLastIncludedIndex(), meta.getLastIncludedTerm()), conf, oldConf); this.configManager.setSnapshot(entry); @@ -670,6 +678,14 @@ private Configuration oldConfFromMeta(final SnapshotMeta meta) { peer.parse(meta.getOldLearners(i)); oldConf.addLearner(peer); } + // load old factor from meta + oldConf.setReadFactor(meta.getOldReadFactor()); + oldConf.setWriteFactor(meta.getOldWriteFactor()); + oldConf.setEnableFlexible(meta.getIsEnableFlexible()); + if (meta.hasOldQuorum()) { + Quorum oldQuorum = new Quorum(meta.getOldQuorum().getW(), meta.getOldQuorum().getR()); + oldConf.setQuorum(oldQuorum); + } return oldConf; } @@ -685,6 +701,12 @@ private Configuration confFromMeta(final SnapshotMeta meta) { peer.parse(meta.getLearners(i)); conf.addLearner(peer); } + conf.setEnableFlexible(meta.getIsEnableFlexible()); + if (meta.hasReadFactor() || meta.hasWriteFactor()) { + conf.setReadFactor(meta.getReadFactor()); + conf.setWriteFactor(meta.getWriteFactor()); + } + conf.setQuorum(new Quorum(meta.getQuorum().getW(), meta.getQuorum().getR())); return conf; } @@ -984,7 +1006,7 @@ private boolean reset(final long nextLogIndex) { private void unsafeTruncateSuffix(final long lastIndexKept, final Lock lock) { if (lastIndexKept < this.appliedId.getIndex()) { LOG.error("FATAL ERROR: Can't truncate logs before appliedId={}, lastIndexKept={}", this.appliedId, - lastIndexKept); + lastIndexKept); return; } @@ -1107,8 +1129,8 @@ private long notifyOnNewLog(final long expectedLastLogIndex, final WaitMeta wm) } long waitId = this.nextWaitId++; if (waitId < 0) { - // Valid waitId starts from 1, skip 0. - waitId = this.nextWaitId = 1; + // Valid waitId starts from 1, skip 0. + waitId = this.nextWaitId = 1; } this.waitMap.put(waitId, wm); return waitId; diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/RocksDBLogStorage.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/RocksDBLogStorage.java index 738628246..a574e8983 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/RocksDBLogStorage.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/storage/impl/RocksDBLogStorage.java @@ -21,10 +21,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.util.*; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.ColumnFamilyDescriptor; @@ -253,9 +255,18 @@ private void load(final ConfigurationManager confManager) { if (entry.getType() == EntryType.ENTRY_TYPE_CONFIGURATION) { final ConfigurationEntry confEntry = new ConfigurationEntry(); confEntry.setId(new LogId(entry.getId().getIndex(), entry.getId().getTerm())); - confEntry.setConf(new Configuration(entry.getPeers(), entry.getLearners())); + Quorum quorum = new Quorum(entry.getQuorum().getW(), entry.getQuorum().getR()); + Configuration conf = new Configuration(entry.getPeers(), entry.getLearners(), quorum, + entry.getWriteFactor(), entry.getReadFactor(), entry.getEnableFlexible()); + confEntry.setConf(conf); if (entry.getOldPeers() != null) { - confEntry.setOldConf(new Configuration(entry.getOldPeers(), entry.getOldLearners())); + Quorum oldQuorum = null; + if(Objects.nonNull(entry.getOldQuorum())) { + oldQuorum = new Quorum(entry.getOldQuorum().getW(), entry.getOldQuorum().getR()); + } + Configuration oldConf = new Configuration(entry.getOldPeers(), entry.getOldLearners(), + oldQuorum, entry.getOldWriteFactor(), entry.getOldReadFactor(), entry.getEnableFlexible()); + confEntry.setOldConf(oldConf); } if (confManager != null) { confManager.add(confEntry); diff --git a/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Bits.java b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Bits.java index b2947dbb9..0e97723cc 100644 --- a/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Bits.java +++ b/jraft-core/src/main/java/com/alipay/sofa/jraft/util/Bits.java @@ -47,4 +47,12 @@ public static short getShort(final byte[] b, final int off) { public static void putLong(final byte[] b, final int off, final long val) { HeapByteBufUtil.setLong(b, off, val); } + + public static void putBoolean(final byte[] b, final int off, final int val) { + HeapByteBufUtil.setByte(b, off, val); + } + + public static Byte getBoolean(final byte[] b, final int off) { + return HeapByteBufUtil.getByte(b, off); + } } diff --git a/jraft-core/src/main/resources/cli.proto b/jraft-core/src/main/resources/cli.proto index 93a464e65..c3381e858 100644 --- a/jraft-core/src/main/resources/cli.proto +++ b/jraft-core/src/main/resources/cli.proto @@ -105,3 +105,16 @@ message LearnersOpResponse { repeated string new_learners = 2; optional ErrorResponse errorResponse = 99; } + +message ResetFactorRequest { + required string group_id = 1; + required string leader_id = 2; + optional int32 read_factor = 3; + optional int32 write_factor = 4; +} + +message ResetFactorResponse { + optional int32 read_factor = 1; + optional int32 write_factor = 2; + optional ErrorResponse errorResponse = 99; +} diff --git a/jraft-core/src/main/resources/log.proto b/jraft-core/src/main/resources/log.proto index b410fcac5..0bc26ef04 100644 --- a/jraft-core/src/main/resources/log.proto +++ b/jraft-core/src/main/resources/log.proto @@ -7,6 +7,11 @@ import "enum.proto"; option java_package="com.alipay.sofa.jraft.entity.codec.v2"; option java_outer_classname = "LogOutter"; +message Quorum{ + optional int32 w = 1; + optional int32 r = 2; +} + message PBLogEntry { required EntryType type = 1; required int64 term = 2; @@ -17,4 +22,11 @@ message PBLogEntry { optional int64 checksum = 7; repeated bytes learners = 8; repeated bytes old_learners = 9; + optional int32 read_factor = 10; + optional int32 write_factor = 11; + optional int32 old_read_factor = 12; + optional int32 old_write_factor = 13; + optional bool is_enable_flexible = 14; + optional Quorum quorum = 15; + optional Quorum old_quorum = 16; }; diff --git a/jraft-core/src/main/resources/raft.desc b/jraft-core/src/main/resources/raft.desc index 6b0a544c0..eccb17ee7 100644 Binary files a/jraft-core/src/main/resources/raft.desc and b/jraft-core/src/main/resources/raft.desc differ diff --git a/jraft-core/src/main/resources/raft.proto b/jraft-core/src/main/resources/raft.proto index 040c2dee8..c57feef7a 100644 --- a/jraft-core/src/main/resources/raft.proto +++ b/jraft-core/src/main/resources/raft.proto @@ -1,10 +1,11 @@ -syntax="proto2"; +syntax = "proto2"; package jraft; import "enum.proto"; +import "log.proto"; -option java_package="com.alipay.sofa.jraft.entity"; +option java_package = "com.alipay.sofa.jraft.entity"; option java_outer_classname = "RaftOutter"; @@ -20,6 +21,13 @@ message EntryMeta { optional int64 checksum = 6; repeated string learners = 7; repeated string old_learners = 8; + optional int32 read_factor = 9; + optional int32 write_factor = 10; + optional int32 old_read_factor = 11; + optional int32 old_write_factor = 12; + optional bool isEnableFlexible = 13; + optional Quorum quorum = 14; + optional Quorum old_quorum = 15; }; message SnapshotMeta { @@ -29,4 +37,11 @@ message SnapshotMeta { repeated string old_peers = 4; repeated string learners = 5; repeated string old_learners = 6; + optional int32 read_factor = 7; + optional int32 write_factor = 8; + optional int32 old_read_factor = 9; + optional int32 old_write_factor = 10; + optional bool isEnableFlexible = 11; + optional Quorum quorum = 12; + optional Quorum old_quorum = 13; } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/conf/ConfigurationTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/conf/ConfigurationTest.java index cb48fceca..b8fb89194 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/conf/ConfigurationTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/conf/ConfigurationTest.java @@ -24,6 +24,7 @@ import org.junit.Test; import com.alipay.sofa.jraft.JRaftUtils; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.PeerId; public class ConfigurationTest { @@ -37,14 +38,15 @@ public void testToStringParseStuff() { assertTrue(peer.toString().startsWith("localhost:80")); } assertFalse(conf.isEmpty()); - assertEquals(confStr, conf.toString()); + assertEquals(confStr, conf.toBasicString()); final Configuration newConf = new Configuration(); - assertTrue(newConf.parse(conf.toString())); + assertTrue(newConf.parse(conf.toBasicString())); + newConf.setQuorum(BallotFactory.buildMajorityQuorum(newConf.size())); assertEquals(3, newConf.getPeerSet().size()); assertTrue(newConf.contains(new PeerId("localhost", 8081))); assertTrue(newConf.contains(new PeerId("localhost", 8082))); assertTrue(newConf.contains(new PeerId("localhost", 8083))); - assertEquals(confStr, newConf.toString()); + assertEquals(confStr, newConf.toBasicString()); assertEquals(conf.hashCode(), newConf.hashCode()); assertEquals(conf, newConf); } @@ -60,14 +62,15 @@ public void testToStringParseStuffWithPriority() { assertEquals(1, peer.getIdx()); } assertFalse(conf.isEmpty()); - assertEquals(confStr, conf.toString()); + assertEquals(confStr, conf.toBasicString()); final Configuration newConf = new Configuration(); - assertTrue(newConf.parse(conf.toString())); + assertTrue(newConf.parse(conf.toBasicString())); + newConf.setQuorum(BallotFactory.buildMajorityQuorum(newConf.size())); assertEquals(3, newConf.getPeerSet().size()); assertTrue(newConf.contains(new PeerId("localhost", 8081, 1, 100))); assertTrue(newConf.contains(new PeerId("localhost", 8082, 1, 100))); assertTrue(newConf.contains(new PeerId("localhost", 8083, 1, 100))); - assertEquals(confStr, newConf.toString()); + assertEquals(confStr, newConf.toBasicString()); assertEquals(conf.hashCode(), newConf.hashCode()); assertEquals(conf, newConf); } @@ -86,14 +89,15 @@ public void testToStringParseStuffWithPriorityAndNone() { } } assertFalse(conf.isEmpty()); - assertEquals(confStr, conf.toString()); + assertEquals(confStr, conf.toBasicString()); final Configuration newConf = new Configuration(); - assertTrue(newConf.parse(conf.toString())); + assertTrue(newConf.parse(conf.toBasicString())); + newConf.setQuorum(BallotFactory.buildMajorityQuorum(newConf.size())); assertEquals(3, newConf.getPeerSet().size()); assertTrue(newConf.contains(new PeerId("localhost", 8081))); assertTrue(newConf.contains(new PeerId("localhost", 8082))); assertTrue(newConf.contains(new PeerId("localhost", 8083, 1, 100))); - assertEquals(confStr, newConf.toString()); + assertEquals(confStr, newConf.toBasicString()); assertEquals(conf.hashCode(), newConf.hashCode()); assertEquals(conf, newConf); } @@ -103,7 +107,7 @@ public void testLearnerStuff() { final String confStr = "localhost:8081,localhost:8082,localhost:8083"; final Configuration conf = JRaftUtils.getConfiguration(confStr); assertEquals(3, conf.size()); - assertEquals(confStr, conf.toString()); + assertEquals(confStr, conf.toBasicString()); assertTrue(conf.isValid()); PeerId learner1 = new PeerId("192.168.1.1", 8081); @@ -117,13 +121,13 @@ public void testLearnerStuff() { assertTrue(conf.getLearners().contains(learner2)); String newConfStr = "localhost:8081,localhost:8082,localhost:8083,192.168.1.1:8081/learner,192.168.1.2:8081/learner"; - assertEquals(newConfStr, conf.toString()); + assertEquals(newConfStr, conf.toBasicString()); assertTrue(conf.isValid()); final Configuration newConf = JRaftUtils.getConfiguration(newConfStr); assertEquals(newConf, conf); assertEquals(2, newConf.getLearners().size()); - assertEquals(newConfStr, newConf.toString()); + assertEquals(newConfStr, newConf.toBasicString()); assertTrue(newConf.isValid()); // Also adds localhost:8081 as learner @@ -139,7 +143,8 @@ public void testCopy() { assertEquals(conf, copied); assertNotSame(conf, copied); assertEquals(copied.size(), 3); - assertEquals("localhost:8081,localhost:8082,localhost:8083", copied.toString()); + assertEquals("localhost:8081,localhost:8082,localhost:8083,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", + copied.toString()); final PeerId newPeer = new PeerId("localhost", 8084); conf.addPeer(newPeer); @@ -166,7 +171,7 @@ public void testDiff() { final Configuration included = new Configuration(); final Configuration excluded = new Configuration(); conf1.diff(conf2, included, excluded); - assertEquals("localhost:8082", included.toString()); - assertEquals("localhost:8085,localhost:8086", excluded.toString()); + assertEquals("localhost:8082,isEnableFlexible:false", included.toString()); + assertEquals("localhost:8085,localhost:8086,isEnableFlexible:false", excluded.toString()); } } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/BallotBoxTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/BallotBoxTest.java index e5fccfc7c..0f9fa5f23 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/BallotBoxTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/BallotBoxTest.java @@ -16,6 +16,8 @@ */ package com.alipay.sofa.jraft.core; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.entity.BallotFactory; import org.junit.After; import org.junit.Before; import org.junit.Test; diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java index c9a7d1479..cf58546a6 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/CliServiceTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import com.alipay.sofa.jraft.entity.BallotFactory; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Before; @@ -99,6 +100,7 @@ public void setup() throws Exception { this.cliService = new CliServiceImpl(); this.conf = new Configuration(peers, learners); + this.conf.setQuorum(BallotFactory.buildMajorityQuorum(peers.size())); assertTrue(this.cliService.init(new CliOptions())); } @@ -274,7 +276,9 @@ public void testChangePeers() throws Exception { assertNotNull(oldLeaderNode); final PeerId oldLeader = oldLeaderNode.getNodeId().getPeerId(); assertNotNull(oldLeader); - assertTrue(this.cliService.changePeers(this.groupId, this.conf, new Configuration(newPeers)).isOk()); + Configuration conf = new Configuration(newPeers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(newPeers.size())); + assertTrue(this.cliService.changePeers(this.groupId, this.conf, conf).isOk()); this.cluster.waitLeader(); final PeerId newLeader = this.cluster.getLeader().getNodeId().getPeerId(); assertNotEquals(oldLeader, newLeader); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/FlexibleRaftTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/FlexibleRaftTest.java new file mode 100644 index 000000000..97b86416d --- /dev/null +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/FlexibleRaftTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.core; + +import com.alipay.sofa.jraft.CliService; +import com.alipay.sofa.jraft.NodeManager; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.RouteTable; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.BallotFactory; +import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.option.CliOptions; +import com.alipay.sofa.jraft.test.TestUtils; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +import java.io.File; +import java.util.LinkedHashSet; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * @author Akai + */ +public class FlexibleRaftTest { + private String dataPath; + private TestCluster cluster; + private final String groupId = "FlexibleRaftTest"; + + private Integer readFactor; + private Integer writeFactor; + private Boolean enableFlexible; + private CliService cliService; + + private Configuration conf; + + @Rule + public TestName testName = new TestName(); + private static final int LEARNER_PORT_STEP = 100; + + @Before + public void setup() throws Exception { + System.out.println(">>>>>>>>>>>>>>> Start test method: " + this.testName.getMethodName()); + this.dataPath = TestUtils.mkTempDir(); + this.readFactor = 4; + this.writeFactor = 6; + this.enableFlexible = true; + FileUtils.forceMkdir(new File(this.dataPath)); + assertEquals(NodeImpl.GLOBAL_NUM_NODES.get(), 0); + final List peers = TestUtils.generatePeers(5); + + final LinkedHashSet learners = new LinkedHashSet<>(); + //2 learners + for (int i = 0; i < 2; i++) { + learners.add(new PeerId(TestUtils.getMyIp(), TestUtils.INIT_PORT + LEARNER_PORT_STEP + i)); + } + + this.cluster = new TestCluster(this.groupId, this.dataPath, peers, learners, 300); + for (final PeerId peer : peers) { + this.cluster.startWithFlexible(peer.getEndpoint(), this.readFactor, this.writeFactor); + } + + for (final PeerId peer : learners) { + this.cluster.startLearnerWithFlexible(peer, this.readFactor, this.writeFactor); + } + + this.cluster.waitLeader(); + Thread.sleep(1000); + this.cliService = new CliServiceImpl(); + Quorum quorum = BallotFactory.buildFlexibleQuorum(this.readFactor, this.writeFactor, peers.size()); + this.conf = new Configuration(peers, learners, quorum, this.readFactor, this.writeFactor, this.enableFlexible); + + assertTrue(this.cliService.init(new CliOptions())); + } + + @Test + public void testResetFactor() { + + Integer changeReadFactor = 8; + Integer changeWriteFactor = 2; + + assertTrue(this.cliService.resetFactor(this.groupId, this.conf, changeReadFactor, changeWriteFactor).isOk()); + } + + @After + public void teardown() throws Exception { + this.cliService.shutdown(); + this.cluster.stopAll(); + if (NodeImpl.GLOBAL_NUM_NODES.get() > 0) { + Thread.sleep(1000); + assertEquals(NodeImpl.GLOBAL_NUM_NODES.get(), 0); + } + FileUtils.deleteDirectory(new File(this.dataPath)); + NodeManager.getInstance().clear(); + RouteTable.getInstance().reset(); + System.out.println(">>>>>>>>>>>>>>> End test method: " + this.testName.getMethodName()); + } +} diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java index 213bb10d0..bb28f0571 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java @@ -33,8 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import com.alipay.sofa.jraft.util.ThreadPoolsFactory; -import com.codahale.metrics.MetricRegistry; +import com.alipay.sofa.jraft.entity.BallotFactory; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.AfterClass; @@ -570,6 +569,7 @@ public void testLeaderTransferWithReplicatorStateListener() throws Exception { assertTrue(leader.transferLeadershipTo(targetPeer).isOk()); Thread.sleep(1000); cluster.waitLeader(); + Thread.sleep(1000); assertEquals(2, this.startedCounter.get()); for (Node node : cluster.getNodes()) { @@ -748,7 +748,7 @@ public void testResetLearners() throws Exception { cluster.waitLeader(); Node leader = cluster.getLeader(); - + Thread.sleep(500); assertEquals(3, leader.listAlivePeers().size()); assertEquals(3, leader.listAliveLearners().size()); @@ -1163,7 +1163,9 @@ public void testRemoveLeaderWithPriority() throws Exception { LOG.info("Remove old leader {}", oldLeader); CountDownLatch latch = new CountDownLatch(1); leader.removePeer(oldLeader, new ExpectClosure(latch)); + Thread.sleep(500); waitLatch(latch); + Thread.sleep(500); assertEquals(60, leader.getNodeTargetPriority()); // stop and clean old leader @@ -1203,7 +1205,6 @@ public void testTripleNodesV1V2Codec() throws Exception { assertEquals(3, leader.listPeers().size()); // apply tasks to leader this.sendTestTaskAndWait(leader); - { final ByteBuffer data = ByteBuffer.wrap("no closure".getBytes()); final Task task = new Task(data, null); @@ -1238,7 +1239,6 @@ public void onCommitted() { cluster.ensureSame(-1); assertEquals(2, cluster.getFollowers().size()); - // transfer the leader to v1 codec peer assertTrue(leader.transferLeadershipTo(peers.get(2)).isOk()); cluster.waitLeader(); @@ -1249,13 +1249,11 @@ public void onCommitted() { this.sendTestTaskAndWait(leader); cluster.ensureSame(); cluster.stopAll(); - // start the cluster with v2 codec, should work final TestCluster newCluster = new TestCluster("unittest", this.dataPath, peers); for (int i = 0; i < peers.size(); i++) { assertTrue(newCluster.start(peers.get(i).getEndpoint())); } - // elect leader newCluster.waitLeader(); newCluster.ensureSame(); @@ -1355,6 +1353,7 @@ public void testChecksum() throws Exception { @Test public void testReadIndex() throws Exception { + Thread.sleep(1000); final List peers = TestUtils.generatePeers(3); final TestCluster cluster = new TestCluster("unittest", this.dataPath, peers); @@ -1723,8 +1722,9 @@ public void testJoinNodes() throws Exception { CountDownLatch latch = new CountDownLatch(1); peers.add(peer1); leader.addPeer(peer1, new ExpectClosure(latch)); + Thread.sleep(500); waitLatch(latch); - + Thread.sleep(500); cluster.ensureSame(-1); assertEquals(2, cluster.getFsms().size()); for (final MockStateMachine fsm : cluster.getFsms()) { @@ -1963,7 +1963,9 @@ public void testSetPeer1() throws Exception { final List peers = new ArrayList<>(); peers.add(bootPeer); // reset peers from empty - assertTrue(nodes.get(0).resetPeers(new Configuration(peers)).isOk()); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(peers.size())); + assertTrue(nodes.get(0).resetPeers(conf).isOk()); cluster.waitLeader(); assertNotNull(cluster.getLeader()); @@ -2019,10 +2021,14 @@ public void testSetPeer2() throws Exception { newPeers.add(new PeerId(leaderAddr, 0)); // new peers equal to current conf - assertTrue(leader.resetPeers(new Configuration(peers)).isOk()); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(peers.size())); + assertTrue(leader.resetPeers(conf).isOk()); // set peer when quorum die LOG.warn("Set peers to {}", leaderAddr); - assertTrue(leader.resetPeers(new Configuration(newPeers)).isOk()); + Configuration newConf = new Configuration(newPeers); + newConf.setQuorum(BallotFactory.buildMajorityQuorum(newPeers.size())); + assertTrue(leader.resetPeers(newConf).isOk()); cluster.waitLeader(); leader = cluster.getLeader(); @@ -2931,6 +2937,7 @@ public void testFollowerStartStopFollowing() throws Exception { // assert follow times final List firstFollowers = cluster.getFollowers(); assertEquals(4, firstFollowers.size()); + Thread.sleep(500); for (final Node node : firstFollowers) { assertEquals(1, ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes()); assertEquals(0, ((MockStateMachine) node.getOptions().getFsm()).getOnStopFollowingTimes()); @@ -3221,8 +3228,9 @@ public void testChangePeersAddMultiNodes() throws Exception { done.reset(); // works leader.changePeers(conf, done); + Thread.sleep(500); assertTrue(done.await().isOk()); - + Thread.sleep(500); assertTrue(cluster.ensureSame()); assertEquals(3, cluster.getFsms().size()); for (final MockStateMachine fsm : cluster.getFsms()) { @@ -3337,6 +3345,7 @@ private Future startChangePeersThread(final ChangeArg arg) { conf.addPeer(arg.peers.get(i)); } } + conf.setQuorum(BallotFactory.buildMajorityQuorum(conf.size())); if (conf.isEmpty()) { LOG.warn("No peer has been selected"); continue; @@ -3393,7 +3402,10 @@ public void testChangePeersChaosWithSnapshot() throws Exception { cluster.waitLeader(); final SynchronizedClosure done = new SynchronizedClosure(); final Node leader = cluster.getLeader(); - leader.changePeers(new Configuration(peers), done); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(peers.size())); + Thread.sleep(1000); + leader.changePeers(conf, done); final Status st = done.await(); assertTrue(st.getErrorMsg(), st.isOk()); cluster.ensureSame(); @@ -3525,12 +3537,14 @@ public void testChangePeersChaosApplyTasks() throws Exception { cluster.waitLeader(); final SynchronizedClosure done = new SynchronizedClosure(); final Node leader = cluster.getLeader(); - leader.changePeers(new Configuration(peers), done); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(peers.size())); + leader.changePeers(conf, done); try { - Status status = done.await(); - assertTrue(status.getErrorMsg(), status.isOk()); - cluster.ensureSame(); - assertEquals(10, cluster.getFsms().size()); + Status status = done.await(); + assertTrue(status.getErrorMsg(), status.isOk()); + cluster.ensureSame(); + assertEquals(10, cluster.getFsms().size()); for (final MockStateMachine fsm : cluster.getFsms()) { final int logSize = fsm.getLogs().size(); assertTrue("logSize= " + logSize, logSize >= 5000 * threads); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java index aa1bf5e26..63f4795cf 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/core/TestCluster.java @@ -24,6 +24,7 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -31,12 +32,14 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; +import com.alipay.sofa.jraft.util.BufferUtils; import org.apache.commons.io.FileUtils; import com.alipay.sofa.jraft.JRaftServiceFactory; import com.alipay.sofa.jraft.Node; import com.alipay.sofa.jraft.RaftGroupService; import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.PeerId; import com.alipay.sofa.jraft.option.NodeOptions; import com.alipay.sofa.jraft.option.RaftOptions; @@ -141,11 +144,22 @@ public boolean start(final Endpoint addr, final int priority) throws Exception { return this.start(addr, false, 300, false, null, null, priority); } + public boolean startWithFlexible(final Endpoint addr, final Integer readFactor, final Integer writeFactor) + throws Exception { + return this.start(addr, false, 300, false, readFactor, writeFactor, null, null); + } + public boolean startLearner(final PeerId peer) throws Exception { this.learners.add(peer); return this.start(peer.getEndpoint(), false, 300); } + public boolean startLearnerWithFlexible(final PeerId peer, final Integer readFactor, final Integer writeFactor) + throws Exception { + this.learners.add(peer); + return this.start(peer.getEndpoint(), false, 300, false, readFactor, writeFactor, null, null); + } + public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs) throws IOException { return this.start(listenAddr, emptyPeers, snapshotIntervalSecs, false); @@ -237,7 +251,62 @@ public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final nodeOptions.setFsm(fsm); if (!emptyPeers) { - nodeOptions.setInitialConf(new Configuration(this.peers, this.learners)); + Configuration conf = new Configuration(this.peers, this.learners); + conf.setQuorum(BallotFactory.buildMajorityQuorum(conf.size())); + nodeOptions.setInitialConf(conf); + } + final RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(listenAddr); + final RaftGroupService server = new RaftGroupService(this.name, new PeerId(listenAddr, 0), nodeOptions, + rpcServer); + + this.lock.lock(); + try { + if (this.serverMap.put(listenAddr.toString(), server) == null) { + final Node node = server.start(); + + this.fsms.put(new PeerId(listenAddr, 0), fsm); + this.nodes.add((NodeImpl) node); + return true; + } + } finally { + this.lock.unlock(); + } + return false; + } + + public boolean start(final Endpoint listenAddr, final boolean emptyPeers, final int snapshotIntervalSecs, + final boolean enableMetrics, final Integer readFactor, final Integer writeFactor, + final SnapshotThrottle snapshotThrottle, final RaftOptions raftOptions) throws IOException { + + if (this.serverMap.get(listenAddr.toString()) != null) { + return true; + } + + final NodeOptions nodeOptions = new NodeOptions(); + nodeOptions.setElectionTimeoutMs(this.electionTimeoutMs); + nodeOptions.setEnableMetrics(enableMetrics); + nodeOptions.setSnapshotThrottle(snapshotThrottle); + nodeOptions.setSnapshotIntervalSecs(snapshotIntervalSecs); + nodeOptions.setServiceFactory(this.raftServiceFactory); + if (raftOptions != null) { + nodeOptions.setRaftOptions(raftOptions); + } + final String serverDataPath = this.dataPath + File.separator + listenAddr.toString().replace(':', '_'); + FileUtils.forceMkdir(new File(serverDataPath)); + nodeOptions.setLogUri(serverDataPath + File.separator + "logs"); + nodeOptions.setRaftMetaUri(serverDataPath + File.separator + "meta"); + nodeOptions.setSnapshotUri(serverDataPath + File.separator + "snapshot"); + final MockStateMachine fsm = new MockStateMachine(listenAddr); + nodeOptions.setFsm(fsm); + + if (!emptyPeers) { + Configuration initConf = new Configuration(this.peers, this.learners); + nodeOptions.setInitialConf(initConf); + } + + if (Objects.nonNull(readFactor) || Objects.nonNull(writeFactor)) { + nodeOptions.enableFlexibleRaft(true); + nodeOptions.setFactor(readFactor, writeFactor); } final RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(listenAddr); @@ -469,6 +538,8 @@ public boolean ensureSame(final int waitTimes) throws InterruptedException { for (int j = 0; j < first.getLogs().size(); j++) { final ByteBuffer firstData = first.getLogs().get(j); final ByteBuffer fsmData = fsm.getLogs().get(j); + // BufferUtils.flip(firstData); + // BufferUtils.flip(fsmData); if (!firstData.equals(fsmData)) { fsm.unlock(); first.unlock(); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/BallotTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/BallotTest.java index f82388baa..77ea93f49 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/BallotTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/BallotTest.java @@ -16,6 +16,8 @@ */ package com.alipay.sofa.jraft.entity; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.conf.Configuration; import org.junit.Before; import org.junit.Test; @@ -31,7 +33,10 @@ public class BallotTest { @Before public void setup() { this.ballot = new Ballot(); - this.ballot.init(JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083"), null); + Configuration configuration = JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083"); + Quorum quorum = BallotFactory.buildMajorityQuorum(configuration.getPeers().size()); + configuration.setQuorum(quorum); + this.ballot.init(configuration, null); } @Test diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/codec/BaseLogEntryCodecFactoryTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/codec/BaseLogEntryCodecFactoryTest.java index 0e5717732..b8cefa462 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/codec/BaseLogEntryCodecFactoryTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/entity/codec/BaseLogEntryCodecFactoryTest.java @@ -111,6 +111,7 @@ public void testEncodeDecodeWithData() { LogEntry entry = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_NO_OP); entry.setId(new LogId(100, 3)); entry.setData(buf); + entry.setEnableFlexible(false); entry.setPeers(Arrays.asList(new PeerId("localhost", 99, 1), new PeerId("localhost", 100, 2))); assertEquals(buf, entry.getData()); diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/BaseLogStorageTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/BaseLogStorageTest.java index 7152d56d6..70d1ca4ff 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/BaseLogStorageTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/BaseLogStorageTest.java @@ -28,6 +28,8 @@ import com.alipay.sofa.jraft.JRaftUtils; import com.alipay.sofa.jraft.conf.ConfigurationEntry; import com.alipay.sofa.jraft.conf.ConfigurationManager; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.EnumOutter; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; @@ -116,11 +118,13 @@ public void testLoadWithConfigManager() { final LogEntry confEntry1 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry1.setId(new LogId(99, 1)); - confEntry1.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082").listPeers()); + Configuration configuration1 = JRaftUtils.getConfiguration("localhost:8081,localhost:8082"); + BallotFactory.convertConfigToLogEntry(confEntry1, configuration1); final LogEntry confEntry2 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry2.setId(new LogId(100, 2)); - confEntry2.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083").listPeers()); + Configuration configuration2 = JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083"); + BallotFactory.convertConfigToLogEntry(confEntry2, configuration2); assertTrue(this.logStorage.appendEntry(confEntry1)); assertEquals(1, this.logStorage.appendEntries(Arrays.asList(confEntry2))); @@ -133,11 +137,13 @@ public void testLoadWithConfigManager() { ConfigurationEntry conf = this.confManager.getLastConfiguration(); assertNotNull(conf); assertFalse(conf.isEmpty()); - assertEquals("localhost:8081,localhost:8082,localhost:8083", conf.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,localhost:8083,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", + conf.getConf().toString()); conf = this.confManager.get(99); assertNotNull(conf); assertFalse(conf.isEmpty()); - assertEquals("localhost:8081,localhost:8082", conf.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", conf.getConf() + .toString()); } @Test diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/LogManagerTest.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/LogManagerTest.java index e58c8dc88..994d55ab6 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/LogManagerTest.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/storage/impl/LogManagerTest.java @@ -30,10 +30,13 @@ import com.alipay.sofa.jraft.FSMCaller; import com.alipay.sofa.jraft.JRaftUtils; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.conf.ConfigurationEntry; import com.alipay.sofa.jraft.conf.ConfigurationManager; import com.alipay.sofa.jraft.core.NodeMetrics; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.EnumOutter; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; @@ -282,13 +285,15 @@ public void testGetConfiguration() throws Exception { final List entries = new ArrayList<>(2); final LogEntry confEntry1 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry1.setId(new LogId(0, 1)); - confEntry1.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082").listPeers()); + Configuration conf1 = JRaftUtils.getConfiguration("localhost:8081,localhost:8082"); + BallotFactory.convertConfigToLogEntry(confEntry1, conf1); final LogEntry confEntry2 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry2.setId(new LogId(0, 2)); - confEntry2.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083").listPeers()); + Configuration conf2 = JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083"); confEntry2.setOldPeers(confEntry1.getPeers()); - + BallotFactory.convertConfigToLogEntry(confEntry2, conf2); + BallotFactory.convertOldConfigToLogOuterEntry(confEntry2, conf1); entries.add(confEntry1); entries.add(confEntry2); @@ -303,12 +308,15 @@ public void run(final Status status) { }); latch.await(); ConfigurationEntry entry = this.logManager.getConfiguration(1); - assertEquals("localhost:8081,localhost:8082", entry.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", entry.getConf() + .toString()); assertTrue(entry.getOldConf().isEmpty()); entry = this.logManager.getConfiguration(2); - assertEquals("localhost:8081,localhost:8082,localhost:8083", entry.getConf().toString()); - assertEquals("localhost:8081,localhost:8082", entry.getOldConf().toString()); + assertEquals("localhost:8081,localhost:8082,localhost:8083,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", + entry.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", entry.getOldConf() + .toString()); } @Test @@ -406,14 +414,20 @@ public void testCheckAndSetConfiguration() throws Exception { assertNull(this.logManager.checkAndSetConfiguration(null)); final ConfigurationEntry entry = new ConfigurationEntry(); entry.setId(new LogId(0, 1)); - entry.setConf(JRaftUtils.getConfiguration("localhost:8081,localhost:8082")); + Configuration conf = JRaftUtils.getConfiguration("localhost:8081,localhost:8082"); + Quorum quorum = BallotFactory.buildMajorityQuorum(conf.size()); + conf.setQuorum(quorum); + conf.setEnableFlexible(false); + entry.setConf(conf); assertSame(entry, this.logManager.checkAndSetConfiguration(entry)); testGetConfiguration(); final ConfigurationEntry lastEntry = this.logManager.checkAndSetConfiguration(entry); assertNotSame(entry, lastEntry); - assertEquals("localhost:8081,localhost:8082,localhost:8083", lastEntry.getConf().toString()); - assertEquals("localhost:8081,localhost:8082", lastEntry.getOldConf().toString()); + assertEquals("localhost:8081,localhost:8082,localhost:8083,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", + lastEntry.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", lastEntry + .getOldConf().toString()); } } diff --git a/jraft-core/src/test/java/com/alipay/sofa/jraft/test/TestUtils.java b/jraft-core/src/test/java/com/alipay/sofa/jraft/test/TestUtils.java index b9102f946..5103dc6bc 100644 --- a/jraft-core/src/test/java/com/alipay/sofa/jraft/test/TestUtils.java +++ b/jraft-core/src/test/java/com/alipay/sofa/jraft/test/TestUtils.java @@ -34,7 +34,10 @@ import java.util.concurrent.ThreadPoolExecutor; import com.alipay.sofa.jraft.JRaftUtils; +import com.alipay.sofa.jraft.Quorum; +import com.alipay.sofa.jraft.conf.Configuration; import com.alipay.sofa.jraft.conf.ConfigurationEntry; +import com.alipay.sofa.jraft.entity.BallotFactory; import com.alipay.sofa.jraft.entity.EnumOutter; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; @@ -66,8 +69,16 @@ public class TestUtils { public static ConfigurationEntry getConfEntry(final String confStr, final String oldConfStr) { ConfigurationEntry entry = new ConfigurationEntry(); - entry.setConf(JRaftUtils.getConfiguration(confStr)); - entry.setOldConf(JRaftUtils.getConfiguration(oldConfStr)); + // set conf + Configuration conf = JRaftUtils.getConfiguration(confStr); + Quorum quorum = BallotFactory.buildMajorityQuorum(conf.size()); + conf.setQuorum(quorum); + entry.setConf(conf); + // set oldConf + Configuration oldConf = JRaftUtils.getConfiguration(oldConfStr); + Quorum oldQuorum = BallotFactory.buildMajorityQuorum(oldConf.size()); + oldConf.setQuorum(oldQuorum); + entry.setOldConf(oldConf); return entry; } diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java index cedd92677..1cb865332 100644 --- a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/counter/CounterServer.java @@ -71,6 +71,7 @@ public CounterServer(final String dataPath, final String groupId, final PeerId s nodeOptions.setRaftMetaUri(dataPath + File.separator + "raft_meta"); // snapshot, optional, generally recommended nodeOptions.setSnapshotUri(dataPath + File.separator + "snapshot"); + // init raft group service framework this.raftGroupService = new RaftGroupService(groupId, serverId, nodeOptions, rpcServer); // start raft node @@ -130,6 +131,7 @@ public static void main(final String[] args) throws IOException { throw new IllegalArgumentException("Fail to parse serverId:" + serverIdStr); } final Configuration initConf = new Configuration(); + if (!initConf.parse(initConfStr)) { throw new IllegalArgumentException("Fail to parse initConf:" + initConfStr); } diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftClosure.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftClosure.java new file mode 100644 index 000000000..d6f0bc228 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftClosure.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft; + +import com.alipay.sofa.jraft.Closure; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter; + +/** + * @author Akai + */ +public abstract class FlexibleRaftClosure implements Closure { + private FlexibleRaftOutter.FlexibleValueResponse valueResponse; + private Operation operation; + + public void setCounterOperation(Operation operation) { + this.operation = operation; + } + + public Operation getFlexibleOperation() { + return operation; + } + + public FlexibleRaftOutter.FlexibleValueResponse getFlexibleValueResponse() { + return valueResponse; + } + + public void setValueResponse(FlexibleRaftOutter.FlexibleValueResponse valueResponse) { + this.valueResponse = valueResponse; + } + + protected void failure(final String errorMsg, final String redirect) { + final FlexibleRaftOutter.FlexibleValueResponse response = FlexibleRaftOutter.FlexibleValueResponse.newBuilder() + .setSuccess(false).setErrorMsg(errorMsg).setRedirect(redirect).build(); + setValueResponse(response); + } + + protected void success(final long value) { + final FlexibleRaftOutter.FlexibleValueResponse response = FlexibleRaftOutter.FlexibleValueResponse.newBuilder() + .setValue(value).setSuccess(true).build(); + setValueResponse(response); + } + +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftService.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftService.java new file mode 100644 index 000000000..214324897 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftService.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft; + +/** + * @author Akai + */ +public interface FlexibleRaftService { + + /** + * Get current value from counter + * Provide consistent reading if {@code readOnlySafe} is true. + */ + void get(final boolean readOnlySafe, final FlexibleRaftClosure closure); + + /** + * Add delta to counter then get value + */ + void incrementAndGet(final long delta, final FlexibleRaftClosure closure); +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftServiceImpl.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftServiceImpl.java new file mode 100644 index 000000000..362797566 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleRaftServiceImpl.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft; + +import com.alipay.remoting.exception.CodecException; +import com.alipay.remoting.serialization.SerializerManager; +import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.closure.ReadIndexClosure; +import com.alipay.sofa.jraft.entity.Task; +import com.alipay.sofa.jraft.error.RaftError; +import com.alipay.sofa.jraft.example.counter.CounterServiceImpl; +import com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftServer.FlexibleRaftServer; +import com.alipay.sofa.jraft.rhea.StoreEngineHelper; +import com.alipay.sofa.jraft.rhea.options.StoreEngineOptions; +import com.alipay.sofa.jraft.util.BytesUtil; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.concurrent.Executor; + +/** + * @author Akai + */ +public class FlexibleRaftServiceImpl implements FlexibleRaftService { + private static final Logger LOG = LoggerFactory.getLogger(CounterServiceImpl.class); + + private final FlexibleRaftServer flexibleRaftServer; + private final Executor readIndexExecutor; + + public FlexibleRaftServiceImpl(FlexibleRaftServer flexibleRaftServer) { + this.flexibleRaftServer = flexibleRaftServer; + this.readIndexExecutor = createReadIndexExecutor(); + } + + private Executor createReadIndexExecutor() { + final StoreEngineOptions opts = new StoreEngineOptions(); + return StoreEngineHelper.createReadIndexExecutor(opts.getReadIndexCoreThreads()); + } + + @Override + public void get(final boolean readOnlySafe, final FlexibleRaftClosure closure) { + if(!readOnlySafe){ + closure.success(getValue()); + closure.run(Status.OK()); + return; + } + + this.flexibleRaftServer.getNode().readIndex(BytesUtil.EMPTY_BYTES, new ReadIndexClosure() { + @Override + public void run(Status status, long index, byte[] reqCtx) { + if(status.isOk()){ + closure.success(getValue()); + closure.run(Status.OK()); + return; + } + FlexibleRaftServiceImpl.this.readIndexExecutor.execute(() -> { + if(isLeader()){ + LOG.debug("Fail to get value with 'ReadIndex': {}, try to applying to the state machine.", status); + applyOperation(Operation.createGet(), closure); + }else { + handlerNotLeaderError(closure); + } + }); + } + }); + } + + private boolean isLeader() { + return this.flexibleRaftServer.getFsm().isLeader(); + } + + private long getValue() { + return this.flexibleRaftServer.getFsm().getValue(); + } + + private String getRedirect() { + return this.flexibleRaftServer.redirect().getRedirect(); + } + + @Override + public void incrementAndGet(final long delta, final FlexibleRaftClosure closure) { + applyOperation(Operation.createIncrement(delta), closure); + } + + private void applyOperation(final Operation op, final FlexibleRaftClosure closure) { + if (!isLeader()) { + handlerNotLeaderError(closure); + return; + } + + try { + closure.setCounterOperation(op); + final Task task = new Task(); + task.setData(ByteBuffer.wrap(SerializerManager.getSerializer(SerializerManager.Hessian2).serialize(op))); + task.setDone(closure); + this.flexibleRaftServer.getNode().apply(task); + } catch (CodecException e) { + String errorMsg = "Fail to encode CounterOperation"; + LOG.error(errorMsg, e); + closure.failure(errorMsg, StringUtils.EMPTY); + closure.run(new Status(RaftError.EINTERNAL, errorMsg)); + } + } + + private void handlerNotLeaderError(final FlexibleRaftClosure closure) { + closure.failure("Not leader.", getRedirect()); + closure.run(new Status(RaftError.EPERM, "Not leader")); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleStateMachine.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleStateMachine.java new file mode 100644 index 000000000..562ae6e31 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/FlexibleStateMachine.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft; + +import com.alipay.remoting.exception.CodecException; +import com.alipay.remoting.serialization.SerializerManager; +import com.alipay.sofa.jraft.Closure; +import com.alipay.sofa.jraft.Iterator; +import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.core.StateMachineAdapter; +import com.alipay.sofa.jraft.error.RaftError; +import com.alipay.sofa.jraft.error.RaftException; +import com.alipay.sofa.jraft.example.counter.CounterOperation; +import com.alipay.sofa.jraft.example.flexibleRaft.snapshot.FlexibleSnapshotFile; +import com.alipay.sofa.jraft.storage.snapshot.SnapshotReader; +import com.alipay.sofa.jraft.storage.snapshot.SnapshotWriter; +import com.alipay.sofa.jraft.util.NamedThreadFactory; +import com.alipay.sofa.jraft.util.ThreadPoolUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; + +import static com.alipay.sofa.jraft.example.counter.CounterOperation.GET; +import static com.alipay.sofa.jraft.example.counter.CounterOperation.INCREMENT; + +/** + * @author Akai + */ +public class FlexibleStateMachine extends StateMachineAdapter { + private static final Logger LOG = LoggerFactory.getLogger(FlexibleStateMachine.class); + private static ThreadPoolExecutor executor = ThreadPoolUtil + .newBuilder() + .poolName("JRAFT_TEST_EXECUTOR") + .enableMetric(true) + .coreThreads(3) + .maximumThreads(5) + .keepAliveSeconds(60L) + .workQueue(new SynchronousQueue<>()) + .threadFactory( + new NamedThreadFactory("JRaft-Test-Executor-", true)).build(); + /** + * Counter value + */ + private final AtomicLong value = new AtomicLong(0); + /** + * Leader term + */ + private final AtomicLong leaderTerm = new AtomicLong(-1); + + public boolean isLeader() { + return this.leaderTerm.get() > 0; + } + + /** + * Returns current value. + */ + public long getValue() { + return this.value.get(); + } + + @Override + public void onApply(final Iterator iter) { + while (iter.hasNext()) { + long current = 0; + Operation operation = null; + + FlexibleRaftClosure closure = null; + if (iter.done() != null) { + // This task is applied by this node, get value from closure to avoid additional parsing. + closure = (FlexibleRaftClosure) iter.done(); + operation = closure.getFlexibleOperation(); + } else { + // Have to parse FetchAddRequest from this user log. + final ByteBuffer data = iter.getData(); + try { + operation = SerializerManager.getSerializer(SerializerManager.Hessian2).deserialize(data.array(), + CounterOperation.class.getName()); + } catch (final CodecException e) { + LOG.error("Fail to decode IncrementAndGetRequest", e); + } + // follower ignore read operation + if (operation != null && operation.isReadOp()) { + iter.next(); + continue; + } + } + if (operation != null) { + switch (operation.getOp()) { + case GET: + current = this.value.get(); + LOG.info("Get value={} at logIndex={}", current, iter.getIndex()); + break; + case INCREMENT: + final long delta = operation.getDelta(); + final long prev = this.value.get(); + current = this.value.addAndGet(delta); + LOG.info("Added value={} by delta={} at logIndex={}", prev, delta, iter.getIndex()); + break; + } + + if (closure != null) { + closure.success(current); + closure.run(Status.OK()); + } + } + iter.next(); + } + } + + @Override + public void onSnapshotSave(final SnapshotWriter writer, final Closure done) { + final long currVal = this.value.get(); + executor.submit(() -> { + final FlexibleSnapshotFile snapshot = new FlexibleSnapshotFile(writer.getPath() + File.separator + "data"); + if (snapshot.save(currVal)) { + if (writer.addFile("data")) { + done.run(Status.OK()); + } else { + done.run(new Status(RaftError.EIO, "Fail to add file to writer")); + } + } else { + done.run(new Status(RaftError.EIO, "Fail to save counter snapshot %s", snapshot.getPath())); + } + }); + } + + @Override + public void onError(final RaftException e) { + LOG.error("Raft error: {}", e, e); + } + + @Override + public boolean onSnapshotLoad(final SnapshotReader reader) { + if (isLeader()) { + LOG.warn("Leader is not supposed to load snapshot"); + return false; + } + if (reader.getFileMeta("data") == null) { + LOG.error("Fail to find data file in {}", reader.getPath()); + return false; + } + final FlexibleSnapshotFile snapshot = new FlexibleSnapshotFile(reader.getPath() + File.separator + "data"); + try { + this.value.set(snapshot.load()); + return true; + } catch (final IOException e) { + LOG.error("Fail to load snapshot from {}", snapshot.getPath()); + return false; + } + + } + + @Override + public void onLeaderStart(final long term) { + this.leaderTerm.set(term); + super.onLeaderStart(term); + + } + + @Override + public void onLeaderStop(final Status status) { + this.leaderTerm.set(-1); + super.onLeaderStop(status); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/Operation.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/Operation.java new file mode 100644 index 000000000..199364d1a --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/Operation.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft; + +import java.io.Serializable; + +/** + * @author Akai + */ +public class Operation implements Serializable { + + private static final long serialVersionUID = -6597003954824547295L; + + /** Get value */ + public static final byte GET = 0x01; + /** Increment and get value */ + public static final byte INCREMENT = 0x02; + + private byte op; + private long delta; + + public static Operation createGet() { + return new Operation(GET); + } + + public static Operation createIncrement(final long delta) { + return new Operation(INCREMENT, delta); + } + + public Operation(byte op) { + this(op, 0); + } + + public Operation(byte op, long delta) { + this.op = op; + this.delta = delta; + } + + public byte getOp() { + return op; + } + + public long getDelta() { + return delta; + } + + public boolean isReadOp() { + return GET == this.op; + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/AddPeerClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/AddPeerClient.java new file mode 100644 index 000000000..af2ded042 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/AddPeerClient.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftClient; + +import com.alipay.sofa.jraft.CliService; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.core.CliServiceImpl; +import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.option.CliOptions; + +/** + * @author Akai + */ +public class AddPeerClient { + public static void main(String[] args) { + CliService cliService = new CliServiceImpl(); + cliService.init(new CliOptions()); + PeerId peerId = new PeerId("127.0.0.1", 8086); + String groupId = "counter"; + String confStr = "127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083,127.0.0.1:8084,127.0.0.1:8085"; + final Configuration conf = new Configuration(); + if (!conf.parse(confStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + cliService.addPeer(groupId, conf, peerId); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ChangePeersClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ChangePeersClient.java new file mode 100644 index 000000000..1608a725a --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ChangePeersClient.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftClient; + +import com.alipay.sofa.jraft.CliService; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.core.CliServiceImpl; +import com.alipay.sofa.jraft.option.CliOptions; + +public class ChangePeersClient { + public static void main(String[] args) { + CliService cliService = new CliServiceImpl(); + cliService.init(new CliOptions()); + String groupId = "counter"; + String oldConfStr = "127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083,127.0.0.1:8084,127.0.0.1:8085,127.0.0.1:8086"; + String confStr = "127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083,127.0.0.1:8084,127.0.0.1:8085"; + final Configuration conf = new Configuration(); + final Configuration oldConf = new Configuration(); + if (!conf.parse(confStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + if (!conf.parse(oldConfStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + cliService.changePeers(groupId, oldConf, conf); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ReadClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ReadClient.java new file mode 100644 index 000000000..1c2d06581 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ReadClient.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftClient; + +import com.alipay.sofa.jraft.RouteTable; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.error.RemotingException; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleGrpcHelper; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter; +import com.alipay.sofa.jraft.option.CliOptions; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; + +/** + * @author Akai + */ +public class ReadClient { + public static void main(final String[] args) throws Exception { + if (args.length != 2) { + System.out + .println("Usage : java com.alipay.sofa.jraft.example.flexibleRaft.FlexibleCounterClient {groupId} {conf}"); + System.out + .println("Example: java com.alipay.sofa.jraft.example.flexibleRaft.FlexibleCounterClient counter 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"); + System.exit(1); + } + final String groupId = args[0]; + final String confStr = args[1]; + FlexibleGrpcHelper.initGRpc(); + + final Configuration conf = new Configuration(); + if (!conf.parse(confStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + + RouteTable.getInstance().updateConfiguration(groupId, conf); + + final CliClientServiceImpl cliClientService = new CliClientServiceImpl(); + cliClientService.init(new CliOptions()); + + if (!RouteTable.getInstance().refreshLeader(cliClientService, groupId, 1000).isOk()) { + throw new IllegalStateException("Refresh leader failed"); + } + + final PeerId leader = RouteTable.getInstance().selectLeader(groupId); + System.out.println("Leader is " + leader); + final int n = 1; + final long start = System.currentTimeMillis(); + final CountDownLatch latch = new CountDownLatch(n); + read(cliClientService, leader, latch); + latch.await(); + System.out.println(n + " ops, cost : " + (System.currentTimeMillis() - start) + " ms."); + System.exit(0); + } + + private static void read(final CliClientServiceImpl cliClientService, final PeerId leader, CountDownLatch latch) + throws RemotingException, + InterruptedException { + FlexibleRaftOutter.FlexibleGetValueRequest request = FlexibleRaftOutter.FlexibleGetValueRequest.newBuilder() + .setReadOnlySafe(true).build(); + cliClientService.getRpcClient().invokeAsync(leader.getEndpoint(), request, new InvokeCallback() { + + @Override + public void complete(Object result, Throwable err) { + if (err == null) { + latch.countDown(); + System.out.println("Get result:" + result); + } else { + err.printStackTrace(); + } + } + + @Override + public Executor executor() { + return null; + } + }, 5000); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ResetFactorClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ResetFactorClient.java new file mode 100644 index 000000000..f9d284891 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/ResetFactorClient.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftClient; + +import com.alipay.sofa.jraft.CliService; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.core.CliServiceImpl; +import com.alipay.sofa.jraft.option.CliOptions; + +/** + * @author Akai + */ +public class ResetFactorClient { + public static void main(String[] args) { + CliService cliService = new CliServiceImpl(); + cliService.init(new CliOptions()); + //n=5,w=2,r=4 writeFactor=4,readFactor=6 + //n=5,w=4,r=2 writeFactor=8,readFactor=2 + //n=6,w=3,r=4, writeFactor=4,readFactor=6 + //n=6,w=5,r=2, writeFactor=8,readFactor=2 + int writeFactor = 8; + int readFactor = 2; + String groupId = "counter"; + String confStr = "127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083,127.0.0.1:8084,127.0.0.1:8085"; + final Configuration conf = new Configuration(); + if (!conf.parse(confStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + cliService.resetFactor(groupId, conf, readFactor, writeFactor); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/WriteClient.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/WriteClient.java new file mode 100644 index 000000000..c92cab122 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftClient/WriteClient.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftClient; + +import com.alipay.sofa.jraft.RouteTable; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.error.RemotingException; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleGrpcHelper; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter; +import com.alipay.sofa.jraft.option.CliOptions; +import com.alipay.sofa.jraft.rpc.InvokeCallback; +import com.alipay.sofa.jraft.rpc.impl.cli.CliClientServiceImpl; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; + +/** + * @author Akai + */ +public class WriteClient { + public static void main(final String[] args) throws Exception { + if (args.length != 2) { + System.out + .println("Usage : java com.alipay.sofa.jraft.example.flexibleRaft.FlexibleCounterClient {groupId} {conf}"); + System.out + .println("Example: java com.alipay.sofa.jraft.example.flexibleRaft.FlexibleCounterClient counter 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"); + System.exit(1); + } + final String groupId = args[0]; + final String confStr = args[1]; + FlexibleGrpcHelper.initGRpc(); + + final Configuration conf = new Configuration(); + if (!conf.parse(confStr)) { + throw new IllegalArgumentException("Fail to parse conf:" + confStr); + } + + RouteTable.getInstance().updateConfiguration(groupId, conf); + + final CliClientServiceImpl cliClientService = new CliClientServiceImpl(); + cliClientService.init(new CliOptions()); + + if (!RouteTable.getInstance().refreshLeader(cliClientService, groupId, 1000).isOk()) { + throw new IllegalStateException("Refresh leader failed"); + } + + final PeerId leader = RouteTable.getInstance().selectLeader(groupId); + System.out.println("Leader is " + leader); + final int n = 1000; + final CountDownLatch latch = new CountDownLatch(n); + final long start = System.currentTimeMillis(); + for (int i = 0; i < n; i++) { + write(cliClientService, leader, i, latch); + } + latch.await(); + System.out.println(n + " ops, cost : " + (System.currentTimeMillis() - start) + " ms."); + System.exit(0); + } + + private static void write(final CliClientServiceImpl cliClientService, final PeerId leader, final long delta, + CountDownLatch latch) throws RemotingException, InterruptedException { + FlexibleRaftOutter.FlexibleIncrementAndGetRequest request = FlexibleRaftOutter.FlexibleIncrementAndGetRequest + .newBuilder().setDelta(delta).build(); + cliClientService.getRpcClient().invokeAsync(leader.getEndpoint(), request, new InvokeCallback() { + @Override + public void complete(Object result, Throwable err) { + if (err == null) { + latch.countDown(); + System.out.println("incrementAndGet result:" + result); + } else { + err.printStackTrace(); + latch.countDown(); + } + } + + @Override + public Executor executor() { + return null; + } + }, 5000); + } + +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftServer/FlexibleRaftServer.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftServer/FlexibleRaftServer.java new file mode 100644 index 000000000..058424c6d --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/flexibleRaftServer/FlexibleRaftServer.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.flexibleRaftServer; + +import com.alipay.sofa.jraft.Node; +import com.alipay.sofa.jraft.RaftGroupService; +import com.alipay.sofa.jraft.conf.Configuration; +import com.alipay.sofa.jraft.entity.PeerId; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftService; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftServiceImpl; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleStateMachine; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleGetValueRequestProcessor; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleGrpcHelper; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleIncrementAndGetRequestProcessor; +import com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter; +import com.alipay.sofa.jraft.option.NodeOptions; +import com.alipay.sofa.jraft.rpc.RaftRpcServerFactory; +import com.alipay.sofa.jraft.rpc.RpcServer; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; + +/** + * @author Akai + */ +public class FlexibleRaftServer { + private RaftGroupService raftGroupService; + private Node node; + private FlexibleStateMachine fsm; + + public FlexibleRaftServer(final String dataPath, final String groupId, final PeerId serverId, + final NodeOptions nodeOptions) throws IOException { + // init raft data path, it contains log,meta,snapshot + FileUtils.forceMkdir(new File(dataPath)); + + // here use same RPC server for raft and business. It also can be seperated generally + final RpcServer rpcServer = RaftRpcServerFactory.createRaftRpcServer(serverId.getEndpoint()); + // GrpcServer need init marshaller + FlexibleGrpcHelper.initGRpc(); + FlexibleGrpcHelper.setRpcServer(rpcServer); + + // register business processor + FlexibleRaftService flexibleRaftService = new FlexibleRaftServiceImpl(this); + rpcServer.registerProcessor(new FlexibleGetValueRequestProcessor(flexibleRaftService)); + rpcServer.registerProcessor(new FlexibleIncrementAndGetRequestProcessor(flexibleRaftService)); + // init state machine + this.fsm = new FlexibleStateMachine(); + // set fsm to nodeOptions + nodeOptions.setFsm(this.fsm); + // set storage path (log,meta,snapshot) + // log, must + nodeOptions.setLogUri(dataPath + File.separator + "log"); + // meta, must + nodeOptions.setRaftMetaUri(dataPath + File.separator + "raft_meta"); + // snapshot, optional, generally recommended + nodeOptions.setSnapshotUri(dataPath + File.separator + "snapshot"); + // n=5 w=2,r=4 + nodeOptions.enableFlexibleRaft(true); + nodeOptions.setFactor(6, 4); + // init raft group service framework + this.raftGroupService = new RaftGroupService(groupId, serverId, nodeOptions, rpcServer); + // start raft node + this.node = this.raftGroupService.start(); + } + + public FlexibleStateMachine getFsm() { + return this.fsm; + } + + public Node getNode() { + return this.node; + } + + public RaftGroupService RaftGroupService() { + return this.raftGroupService; + } + + /** + * Redirect request to new leader + */ + public FlexibleRaftOutter.FlexibleValueResponse redirect() { + final FlexibleRaftOutter.FlexibleValueResponse.Builder builder = FlexibleRaftOutter.FlexibleValueResponse + .newBuilder().setSuccess(false); + if (this.node != null) { + final PeerId leader = this.node.getLeaderId(); + if (leader != null) { + builder.setRedirect(leader.toString()); + } + } + return builder.build(); + } + + public static void main(final String[] args) throws IOException { + if (args.length != 4) { + System.out + .println("Usage : java com.alipay.sofa.jraft.example.counter.CounterServer {dataPath} {groupId} {serverId} {initConf}"); + System.out + .println("Example: java com.alipay.sofa.jraft.example.counter.CounterServer /tmp/server1 counter 127.0.0.1:8081 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"); + System.exit(1); + } + final String dataPath = args[0]; + final String groupId = args[1]; + final String serverIdStr = args[2]; + final String initConfStr = args[3]; + + final NodeOptions nodeOptions = new NodeOptions(); + // for test, modify some params + // set election timeout to 1s + nodeOptions.setElectionTimeoutMs(1000); + // disable CLI service。 + nodeOptions.setDisableCli(false); + // do snapshot every 30s + nodeOptions.setSnapshotIntervalSecs(30); + // parse server address + final PeerId serverId = new PeerId(); + if (!serverId.parse(serverIdStr)) { + throw new IllegalArgumentException("Fail to parse serverId:" + serverIdStr); + } + final Configuration initConf = new Configuration(); + + if (!initConf.parse(initConfStr)) { + throw new IllegalArgumentException("Fail to parse initConf:" + initConfStr); + } + // set cluster configuration + nodeOptions.setInitialConf(initConf); + + // start raft server + final FlexibleRaftServer flexibleRaftServer = new FlexibleRaftServer(dataPath, groupId, serverId, nodeOptions); + System.out.println("Started counter server at port:" + + flexibleRaftServer.getNode().getNodeId().getPeerId().getPort()); + // GrpcServer need block to prevent process exit + FlexibleGrpcHelper.blockUntilShutdown(); + } + +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGetValueRequestProcessor.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGetValueRequestProcessor.java new file mode 100644 index 000000000..9fec73961 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGetValueRequestProcessor.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.rpc; + +import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftClosure; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftService; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; + +/** + * @author Akai + */ +public class FlexibleGetValueRequestProcessor implements RpcProcessor { + private final FlexibleRaftService flexibleRaftService; + + public FlexibleGetValueRequestProcessor(FlexibleRaftService flexibleRaftService) { + super(); + this.flexibleRaftService = flexibleRaftService; + } + + @Override + public void handleRequest(final RpcContext rpcCtx, final FlexibleRaftOutter.FlexibleGetValueRequest request) { + final FlexibleRaftClosure closure = new FlexibleRaftClosure() { + @Override + public void run(Status status) { + rpcCtx.sendResponse(getFlexibleValueResponse()); + } + }; + + this.flexibleRaftService.get(request.getReadOnlySafe(), closure); + } + + @Override + public String interest() { + return FlexibleRaftOutter.FlexibleGetValueRequest.class.getName(); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGrpcHelper.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGrpcHelper.java new file mode 100644 index 000000000..de4730957 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleGrpcHelper.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.rpc; + +import com.alipay.sofa.jraft.rpc.RpcServer; +import com.alipay.sofa.jraft.util.RpcFactoryHelper; +import com.google.protobuf.Message; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Method; +import java.util.concurrent.TimeUnit; + +/** + * @author Akai + */ +public class FlexibleGrpcHelper { + private static final Logger LOG = LoggerFactory.getLogger(FlexibleGrpcHelper.class); + + public static RpcServer rpcServer; + + public static void initGRpc() { + if ("com.alipay.sofa.jraft.rpc.impl.GrpcRaftRpcFactory".equals(RpcFactoryHelper.rpcFactory().getClass() + .getName())) { + RpcFactoryHelper.rpcFactory().registerProtobufSerializer( + FlexibleRaftOutter.FlexibleGetValueRequest.class.getName(), + FlexibleRaftOutter.FlexibleGetValueRequest.getDefaultInstance()); + RpcFactoryHelper.rpcFactory().registerProtobufSerializer( + FlexibleRaftOutter.FlexibleIncrementAndGetRequest.class.getName(), + FlexibleRaftOutter.FlexibleIncrementAndGetRequest.getDefaultInstance()); + RpcFactoryHelper.rpcFactory().registerProtobufSerializer( + FlexibleRaftOutter.FlexibleValueResponse.class.getName(), + FlexibleRaftOutter.FlexibleValueResponse.getDefaultInstance()); + + try { + Class clazz = Class.forName("com.alipay.sofa.jraft.rpc.impl.MarshallerHelper"); + Method registerRespInstance = clazz.getMethod("registerRespInstance", String.class, Message.class); + registerRespInstance.invoke(null, FlexibleRaftOutter.FlexibleGetValueRequest.class.getName(), + FlexibleRaftOutter.FlexibleValueResponse.getDefaultInstance()); + registerRespInstance.invoke(null, FlexibleRaftOutter.FlexibleIncrementAndGetRequest.class.getName(), + FlexibleRaftOutter.FlexibleValueResponse.getDefaultInstance()); + } catch (Exception e) { + LOG.error("Failed to init grpc server", e); + } + } + } + + public static void setRpcServer(RpcServer rpcServer) { + FlexibleGrpcHelper.rpcServer = rpcServer; + } + + public static void blockUntilShutdown() { + if (rpcServer == null) { + return; + } + if ("com.alipay.sofa.jraft.rpc.impl.GrpcRaftRpcFactory".equals(RpcFactoryHelper.rpcFactory().getClass() + .getName())) { + try { + Method getServer = rpcServer.getClass().getMethod("getServer"); + Object grpcServer = getServer.invoke(rpcServer); + + Method shutdown = grpcServer.getClass().getMethod("shutdown"); + Method awaitTerminationLimit = grpcServer.getClass().getMethod("awaitTermination", long.class, + TimeUnit.class); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + try { + shutdown.invoke(grpcServer); + awaitTerminationLimit.invoke(grpcServer, 30, TimeUnit.SECONDS); + } catch (Exception e) { + // Use stderr here since the logger may have been reset by its JVM shutdown hook. + e.printStackTrace(System.err); + } + } + }); + Method awaitTermination = grpcServer.getClass().getMethod("awaitTermination"); + awaitTermination.invoke(grpcServer); + } catch (Exception e) { + LOG.error("Failed to block grpc server", e); + } + } + } + +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleIncrementAndGetRequestProcessor.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleIncrementAndGetRequestProcessor.java new file mode 100644 index 000000000..2fad3bdaf --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleIncrementAndGetRequestProcessor.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.rpc; + +import com.alipay.sofa.jraft.Status; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftClosure; +import com.alipay.sofa.jraft.example.flexibleRaft.FlexibleRaftService; +import com.alipay.sofa.jraft.rpc.RpcContext; +import com.alipay.sofa.jraft.rpc.RpcProcessor; + +/** + * @author Akai + */ +public class FlexibleIncrementAndGetRequestProcessor implements + RpcProcessor { + + private final FlexibleRaftService flexibleRaftService; + + public FlexibleIncrementAndGetRequestProcessor(FlexibleRaftService flexibleRaftService) { + super(); + this.flexibleRaftService = flexibleRaftService; + } + + @Override + public void handleRequest(final RpcContext rpcCtx, final FlexibleRaftOutter.FlexibleIncrementAndGetRequest request) { + final FlexibleRaftClosure closure = new FlexibleRaftClosure() { + @Override + public void run(Status status) { + rpcCtx.sendResponse(getFlexibleValueResponse()); + } + }; + + this.flexibleRaftService.incrementAndGet(request.getDelta(), closure); + } + + @Override + public String interest() { + return FlexibleRaftOutter.FlexibleIncrementAndGetRequest.class.getName(); + } +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleRaftOutter.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleRaftOutter.java new file mode 100644 index 000000000..c64a38b68 --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/rpc/FlexibleRaftOutter.java @@ -0,0 +1,1991 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: counter.proto + +package com.alipay.sofa.jraft.example.flexibleRaft.rpc; + +public final class FlexibleRaftOutter { + private FlexibleRaftOutter() { + } + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistryLite registry) { + } + + public static void registerAllExtensions(com.google.protobuf.ExtensionRegistry registry) { + registerAllExtensions((com.google.protobuf.ExtensionRegistryLite) registry); + } + + public interface FlexibleGetValueRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.FlexibleGetValueRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * required bool readOnlySafe = 1; + */ + boolean hasReadOnlySafe(); + + /** + * required bool readOnlySafe = 1; + */ + boolean getReadOnlySafe(); + } + + /** + * Protobuf type {@code jraft.FlexibleGetValueRequest} + */ + public static final class FlexibleGetValueRequest extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.FlexibleGetValueRequest) + FlexibleGetValueRequestOrBuilder { + private static final long serialVersionUID = 0L; + + // Use FlexibleGetValueRequest.newBuilder() to construct. + private FlexibleGetValueRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private FlexibleGetValueRequest() { + readOnlySafe_ = false; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private FlexibleGetValueRequest(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + readOnlySafe_ = input.readBool(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return FlexibleRaftOutter.internal_static_jraft_FlexibleGetValueRequest_descriptor; + } + + protected FieldAccessorTable internalGetFieldAccessorTable() { + return FlexibleRaftOutter.internal_static_jraft_FlexibleGetValueRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized(FlexibleGetValueRequest.class, FlexibleGetValueRequest.Builder.class); + } + + private int bitField0_; + public static final int READONLYSAFE_FIELD_NUMBER = 1; + private boolean readOnlySafe_; + + /** + * required bool readOnlySafe = 1; + */ + public boolean hasReadOnlySafe() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required bool readOnlySafe = 1; + */ + public boolean getReadOnlySafe() { + return readOnlySafe_; + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + if (!hasReadOnlySafe()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBool(1, readOnlySafe_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream.computeBoolSize(1, readOnlySafe_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof FlexibleRaftOutter.FlexibleGetValueRequest)) { + return super.equals(obj); + } + FlexibleRaftOutter.FlexibleGetValueRequest other = (FlexibleRaftOutter.FlexibleGetValueRequest) obj; + + boolean result = true; + result = result && (hasReadOnlySafe() == other.hasReadOnlySafe()); + if (hasReadOnlySafe()) { + result = result && (getReadOnlySafe() == other.getReadOnlySafe()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasReadOnlySafe()) { + hash = (37 * hash) + READONLYSAFE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getReadOnlySafe()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.FlexibleGetValueRequest} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.FlexibleGetValueRequest) + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleGetValueRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleGetValueRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest.class, + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + @Override + public boolean getReadOnlySafe() { + return readOnlySafe_; + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + public Builder clear() { + super.clear(); + readOnlySafe_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleGetValueRequest_descriptor; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest getDefaultInstanceForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest + .getDefaultInstance(); + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest build() { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest buildPartial() { + FlexibleRaftOutter.FlexibleGetValueRequest result = new FlexibleRaftOutter.FlexibleGetValueRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.readOnlySafe_ = readOnlySafe_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest) { + return mergeFrom((com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest other) { + if (other == FlexibleRaftOutter.FlexibleGetValueRequest.getDefaultInstance()) + return this; + if (other.hasReadOnlySafe()) { + setReadOnlySafe(other.getReadOnlySafe()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasReadOnlySafe()) { + return false; + } + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest) e + .getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private boolean readOnlySafe_; + + /** + * required bool readOnlySafe = 1; + */ + public boolean hasReadOnlySafe() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required bool readOnlySafe = 1; + */ + + /** + * repeated bool readOnlySafe = 1; + */ + public Builder setReadOnlySafe(boolean value) { + bitField0_ |= 0x00000001; + readOnlySafe_ = value; + onChanged(); + return this; + } + + /** + * repeated bool readOnlySafe = 1; + */ + public Builder clearReadOnlySafe() { + bitField0_ = (bitField0_ & ~0x00000001); + readOnlySafe_ = false; + onChanged(); + return this; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.FlexibleGetValueRequest) + } + + // @@protoc_insertion_point(class_scope:jraft.FlexibleGetValueRequest) + private static final com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest(); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public FlexibleGetValueRequest parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FlexibleGetValueRequest( + input, + extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleGetValueRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface FlexibleIncrementAndGetRequestOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.FlexibleIncrementAndGetRequest) + com.google.protobuf.MessageOrBuilder { + + /** + * required int64 delta = 1; + */ + boolean hasDelta(); + + /** + * required int64 delta = 1; + */ + long getDelta(); + } + + /** + * Protobuf type {@code jraft.FlexibleIncrementAndGetRequest} + */ + public static final class FlexibleIncrementAndGetRequest extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.FlexibleIncrementAndGetRequest) + FlexibleIncrementAndGetRequestOrBuilder { + private static final long serialVersionUID = 0L; + + // Use FlexibleIncrementAndGetRequest.newBuilder() to construct. + private FlexibleIncrementAndGetRequest(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private FlexibleIncrementAndGetRequest() { + delta_ = 0L; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private FlexibleIncrementAndGetRequest(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + delta_ = input.readInt64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleIncrementAndGetRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest.class, + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest.Builder.class); + } + + private int bitField0_; + public static final int DELTA_FIELD_NUMBER = 1; + private long delta_; + + /** + * required int64 delta = 1; + */ + public boolean hasDelta() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required int64 delta = 1; + */ + public long getDelta() { + return delta_; + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + if (!hasDelta()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, delta_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream.computeInt64Size(1, delta_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest)) { + return super.equals(obj); + } + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest other = (com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest) obj; + + boolean result = true; + result = result && (hasDelta() == other.hasDelta()); + if (hasDelta()) { + result = result && (getDelta() == other.getDelta()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasDelta()) { + hash = (37 * hash) + DELTA_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong(getDelta()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.FlexibleIncrementAndGetRequest} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.FlexibleIncrementAndGetRequest) + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleIncrementAndGetRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest.class, + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + public Builder clear() { + super.clear(); + delta_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest getDefaultInstanceForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest + .getDefaultInstance(); + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest build() { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest buildPartial() { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest result = new com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.delta_ = delta_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest) { + return mergeFrom((com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest other) { + if (other == com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest + .getDefaultInstance()) + return this; + if (other.hasDelta()) { + setDelta(other.getDelta()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasDelta()) { + return false; + } + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest) e + .getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private long delta_; + + /** + * required int64 delta = 1; + */ + public boolean hasDelta() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required int64 delta = 1; + */ + public long getDelta() { + return delta_; + } + + /** + * required int64 delta = 1; + */ + public Builder setDelta(long value) { + bitField0_ |= 0x00000001; + delta_ = value; + onChanged(); + return this; + } + + /** + * required int64 delta = 1; + */ + public Builder clearDelta() { + bitField0_ = (bitField0_ & ~0x00000001); + delta_ = 0L; + onChanged(); + return this; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.FlexibleIncrementAndGetRequest) + } + + // @@protoc_insertion_point(class_scope:jraft.FlexibleIncrementAndGetRequest) + private static final com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest(); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public FlexibleIncrementAndGetRequest parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FlexibleIncrementAndGetRequest( + input, + extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleIncrementAndGetRequest getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + public interface FlexibleValueResponseOrBuilder extends + // @@protoc_insertion_point(interface_extends:jraft.FlexibleValueResponse) + com.google.protobuf.MessageOrBuilder { + + /** + * required int64 value = 1; + */ + boolean hasValue(); + + /** + * required int64 value = 1; + */ + long getValue(); + + /** + * required bool success = 2; + */ + boolean hasSuccess(); + + /** + * required bool success = 2; + */ + boolean getSuccess(); + + /** + * optional string redirect = 3; + */ + boolean hasRedirect(); + + /** + * optional string redirect = 3; + */ + java.lang.String getRedirect(); + + /** + * optional string redirect = 3; + */ + com.google.protobuf.ByteString getRedirectBytes(); + + /** + * optional string errorMsg = 4; + */ + boolean hasErrorMsg(); + + /** + * optional string errorMsg = 4; + */ + java.lang.String getErrorMsg(); + + /** + * optional string errorMsg = 4; + */ + com.google.protobuf.ByteString getErrorMsgBytes(); + } + + /** + * Protobuf type {@code jraft.FlexibleValueResponse} + */ + public static final class FlexibleValueResponse extends com.google.protobuf.GeneratedMessageV3 implements + // @@protoc_insertion_point(message_implements:jraft.FlexibleValueResponse) + FlexibleValueResponseOrBuilder { + private static final long serialVersionUID = 0L; + + // Use FlexibleValueResponse.newBuilder() to construct. + private FlexibleValueResponse(com.google.protobuf.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private FlexibleValueResponse() { + value_ = 0L; + success_ = false; + redirect_ = ""; + errorMsg_ = ""; + } + + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private FlexibleValueResponse(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet + .newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + value_ = input.readInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + success_ = input.readBool(); + break; + } + case 26: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000004; + redirect_ = bs; + break; + } + case 34: { + com.google.protobuf.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000008; + errorMsg_ = bs; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException(e).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleValueResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleValueResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse.class, + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse.Builder.class); + } + + private int bitField0_; + public static final int VALUE_FIELD_NUMBER = 1; + private long value_; + + /** + * required int64 value = 1; + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required int64 value = 1; + */ + public long getValue() { + return value_; + } + + public static final int SUCCESS_FIELD_NUMBER = 2; + private boolean success_; + + /** + * required bool success = 2; + */ + public boolean hasSuccess() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required bool success = 2; + */ + public boolean getSuccess() { + return success_; + } + + public static final int REDIRECT_FIELD_NUMBER = 3; + private volatile java.lang.Object redirect_; + + /** + * optional string redirect = 3; + */ + public boolean hasRedirect() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional string redirect = 3; + */ + public java.lang.String getRedirect() { + java.lang.Object ref = redirect_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + redirect_ = s; + } + return s; + } + } + + /** + * optional string redirect = 3; + */ + public com.google.protobuf.ByteString getRedirectBytes() { + java.lang.Object ref = redirect_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + redirect_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + public static final int ERRORMSG_FIELD_NUMBER = 4; + private volatile java.lang.Object errorMsg_; + + /** + * optional string errorMsg = 4; + */ + public boolean hasErrorMsg() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional string errorMsg = 4; + */ + public java.lang.String getErrorMsg() { + java.lang.Object ref = errorMsg_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + errorMsg_ = s; + } + return s; + } + } + + /** + * optional string errorMsg = 4; + */ + public com.google.protobuf.ByteString getErrorMsgBytes() { + java.lang.Object ref = errorMsg_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString.copyFromUtf8((java.lang.String) ref); + errorMsg_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private byte memoizedIsInitialized = -1; + + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) + return true; + if (isInitialized == 0) + return false; + + if (!hasValue()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSuccess()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, value_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, success_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 3, redirect_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + com.google.protobuf.GeneratedMessageV3.writeString(output, 4, errorMsg_); + } + unknownFields.writeTo(output); + } + + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) + return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream.computeInt64Size(1, value_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream.computeBoolSize(2, success_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(3, redirect_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.GeneratedMessageV3.computeStringSize(4, errorMsg_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse)) { + return super.equals(obj); + } + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse other = (com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse) obj; + + boolean result = true; + result = result && (hasValue() == other.hasValue()); + if (hasValue()) { + result = result && (getValue() == other.getValue()); + } + result = result && (hasSuccess() == other.hasSuccess()); + if (hasSuccess()) { + result = result && (getSuccess() == other.getSuccess()); + } + result = result && (hasRedirect() == other.hasRedirect()); + if (hasRedirect()) { + result = result && getRedirect().equals(other.getRedirect()); + } + result = result && (hasErrorMsg() == other.hasErrorMsg()); + if (hasErrorMsg()) { + result = result && getErrorMsg().equals(other.getErrorMsg()); + } + result = result && unknownFields.equals(other.unknownFields); + return result; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasValue()) { + hash = (37 * hash) + VALUE_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashLong(getValue()); + } + if (hasSuccess()) { + hash = (37 * hash) + SUCCESS_FIELD_NUMBER; + hash = (53 * hash) + com.google.protobuf.Internal.hashBoolean(getSuccess()); + } + if (hasRedirect()) { + hash = (37 * hash) + REDIRECT_FIELD_NUMBER; + hash = (53 * hash) + getRedirect().hashCode(); + } + if (hasErrorMsg()) { + hash = (37 * hash) + ERRORMSG_FIELD_NUMBER; + hash = (53 * hash) + getErrorMsg().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(java.nio.ByteBuffer data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(java.nio.ByteBuffer data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseDelimitedFrom(java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseDelimitedWithIOException(PARSER, input, + extensionRegistry); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parseFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return com.google.protobuf.GeneratedMessageV3.parseWithIOException(PARSER, input, extensionRegistry); + } + + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + + /** + * Protobuf type {@code jraft.FlexibleValueResponse} + */ + public static final class Builder extends com.google.protobuf.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:jraft.FlexibleValueResponse) + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleValueResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleValueResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse.class, + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse.Builder.class); + } + + // Construct using com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessageV3.alwaysUseFieldBuilders) { + } + } + + public Builder clear() { + super.clear(); + value_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + success_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + redirect_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + errorMsg_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.internal_static_jraft_FlexibleValueResponse_descriptor; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse getDefaultInstanceForType() { + return com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse + .getDefaultInstance(); + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse build() { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse buildPartial() { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse result = new com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.value_ = value_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.success_ = success_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.redirect_ = redirect_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.errorMsg_ = errorMsg_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder clone() { + return (Builder) super.clone(); + } + + public Builder setField(com.google.protobuf.Descriptors.FieldDescriptor field, java.lang.Object value) { + return (Builder) super.setField(field, value); + } + + public Builder clearField(com.google.protobuf.Descriptors.FieldDescriptor field) { + return (Builder) super.clearField(field); + } + + public Builder clearOneof(com.google.protobuf.Descriptors.OneofDescriptor oneof) { + return (Builder) super.clearOneof(oneof); + } + + public Builder setRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, int index, + java.lang.Object value) { + return (Builder) super.setRepeatedField(field, index, value); + } + + public Builder addRepeatedField(com.google.protobuf.Descriptors.FieldDescriptor field, + java.lang.Object value) { + return (Builder) super.addRepeatedField(field, value); + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse) { + return mergeFrom((com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse other) { + if (other == com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse + .getDefaultInstance()) + return this; + if (other.hasValue()) { + setValue(other.getValue()); + } + if (other.hasSuccess()) { + setSuccess(other.getSuccess()); + } + if (other.hasRedirect()) { + bitField0_ |= 0x00000004; + redirect_ = other.redirect_; + onChanged(); + } + if (other.hasErrorMsg()) { + bitField0_ |= 0x00000008; + errorMsg_ = other.errorMsg_; + onChanged(); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + public final boolean isInitialized() { + if (!hasValue()) { + return false; + } + if (!hasSuccess()) { + return false; + } + return true; + } + + public Builder mergeFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse) e + .getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private long value_; + + /** + * required int64 value = 1; + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + + /** + * required int64 value = 1; + */ + public long getValue() { + return value_; + } + + /** + * required int64 value = 1; + */ + public Builder setValue(long value) { + bitField0_ |= 0x00000001; + value_ = value; + onChanged(); + return this; + } + + /** + * required int64 value = 1; + */ + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000001); + value_ = 0L; + onChanged(); + return this; + } + + private boolean success_; + + /** + * required bool success = 2; + */ + public boolean hasSuccess() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + + /** + * required bool success = 2; + */ + public boolean getSuccess() { + return success_; + } + + /** + * required bool success = 2; + */ + public Builder setSuccess(boolean value) { + bitField0_ |= 0x00000002; + success_ = value; + onChanged(); + return this; + } + + /** + * required bool success = 2; + */ + public Builder clearSuccess() { + bitField0_ = (bitField0_ & ~0x00000002); + success_ = false; + onChanged(); + return this; + } + + private java.lang.Object redirect_ = ""; + + /** + * optional string redirect = 3; + */ + public boolean hasRedirect() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + + /** + * optional string redirect = 3; + */ + public java.lang.String getRedirect() { + java.lang.Object ref = redirect_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + redirect_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * optional string redirect = 3; + */ + public com.google.protobuf.ByteString getRedirectBytes() { + java.lang.Object ref = redirect_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString + .copyFromUtf8((java.lang.String) ref); + redirect_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * optional string redirect = 3; + */ + public Builder setRedirect(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + redirect_ = value; + onChanged(); + return this; + } + + /** + * optional string redirect = 3; + */ + public Builder clearRedirect() { + bitField0_ = (bitField0_ & ~0x00000004); + redirect_ = getDefaultInstance().getRedirect(); + onChanged(); + return this; + } + + /** + * optional string redirect = 3; + */ + public Builder setRedirectBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + redirect_ = value; + onChanged(); + return this; + } + + private java.lang.Object errorMsg_ = ""; + + /** + * optional string errorMsg = 4; + */ + public boolean hasErrorMsg() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + + /** + * optional string errorMsg = 4; + */ + public java.lang.String getErrorMsg() { + java.lang.Object ref = errorMsg_; + if (!(ref instanceof java.lang.String)) { + com.google.protobuf.ByteString bs = (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + errorMsg_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + + /** + * optional string errorMsg = 4; + */ + public com.google.protobuf.ByteString getErrorMsgBytes() { + java.lang.Object ref = errorMsg_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = com.google.protobuf.ByteString + .copyFromUtf8((java.lang.String) ref); + errorMsg_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + /** + * optional string errorMsg = 4; + */ + public Builder setErrorMsg(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + errorMsg_ = value; + onChanged(); + return this; + } + + /** + * optional string errorMsg = 4; + */ + public Builder clearErrorMsg() { + bitField0_ = (bitField0_ & ~0x00000008); + errorMsg_ = getDefaultInstance().getErrorMsg(); + onChanged(); + return this; + } + + /** + * optional string errorMsg = 4; + */ + public Builder setErrorMsgBytes(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + errorMsg_ = value; + onChanged(); + return this; + } + + public final Builder setUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + public final Builder mergeUnknownFields(final com.google.protobuf.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:jraft.FlexibleValueResponse) + } + + // @@protoc_insertion_point(class_scope:jraft.FlexibleValueResponse) + private static final com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse DEFAULT_INSTANCE; + static { + DEFAULT_INSTANCE = new com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse(); + } + + public static com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final com.google.protobuf.Parser PARSER = new com.google.protobuf.AbstractParser() { + public FlexibleValueResponse parsePartialFrom(com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new FlexibleValueResponse( + input, + extensionRegistry); + } + }; + + public static com.google.protobuf.Parser parser() { + return PARSER; + } + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + public com.alipay.sofa.jraft.example.flexibleRaft.rpc.FlexibleRaftOutter.FlexibleValueResponse getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + + } + + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_FlexibleGetValueRequest_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_FlexibleGetValueRequest_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_FlexibleIncrementAndGetRequest_fieldAccessorTable; + private static final com.google.protobuf.Descriptors.Descriptor internal_static_jraft_FlexibleValueResponse_descriptor; + private static final com.google.protobuf.GeneratedMessageV3.FieldAccessorTable internal_static_jraft_FlexibleValueResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static com.google.protobuf.Descriptors.FileDescriptor descriptor; + static { + java.lang.String[] descriptorData = { "\n\026flexible_counter.proto\022\005jraft\"/\n\027Flexi" + + "bleGetValueRequest\022\024\n\014readOnlySafe\030\001 \003(\010" + + "\"/\n\036FlexibleIncrementAndGetRequest\022\r\n\005de" + + "lta\030\001 \002(\003\"[\n\025FlexibleValueResponse\022\r\n\005va" + + "lue\030\001 \002(\003\022\017\n\007success\030\002 \002(\010\022\020\n\010redirect\030\003" + + " \001(\t\022\020\n\010errorMsg\030\004 \001(\tBD\n.com.alipay.sof" + + "a.jraft.example.flexibleRaft.rpcB\022Flexib" + "leRaftOutter" }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors(com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] {}, assigner); + internal_static_jraft_FlexibleGetValueRequest_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_jraft_FlexibleGetValueRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_FlexibleGetValueRequest_descriptor, new java.lang.String[] { "ReadOnlySafe", }); + internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor = getDescriptor().getMessageTypes().get(1); + internal_static_jraft_FlexibleIncrementAndGetRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_FlexibleIncrementAndGetRequest_descriptor, new java.lang.String[] { "Delta", }); + internal_static_jraft_FlexibleValueResponse_descriptor = getDescriptor().getMessageTypes().get(2); + internal_static_jraft_FlexibleValueResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessageV3.FieldAccessorTable( + internal_static_jraft_FlexibleValueResponse_descriptor, new java.lang.String[] { "Value", "Success", + "Redirect", "ErrorMsg", }); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/snapshot/FlexibleSnapshotFile.java b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/snapshot/FlexibleSnapshotFile.java new file mode 100644 index 000000000..0a33c56fd --- /dev/null +++ b/jraft-example/src/main/java/com/alipay/sofa/jraft/example/flexibleRaft/snapshot/FlexibleSnapshotFile.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alipay.sofa.jraft.example.flexibleRaft.snapshot; + +import com.alipay.sofa.jraft.example.counter.snapshot.CounterSnapshotFile; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +/** + * @author Akai + */ +public class FlexibleSnapshotFile { + private static final Logger LOG = LoggerFactory.getLogger(CounterSnapshotFile.class); + + private String path; + + public FlexibleSnapshotFile(String path) { + super(); + this.path = path; + } + + public String getPath() { + return this.path; + } + + /** + * Save value to snapshot file. + */ + public boolean save(final long value) { + try { + FileUtils.writeStringToFile(new File(path), String.valueOf(value)); + return true; + } catch (IOException e) { + LOG.error("Fail to save snapshot", e); + return false; + } + } + + public long load() throws IOException { + final String s = FileUtils.readFileToString(new File(path)); + if (!StringUtils.isBlank(s)) { + return Long.parseLong(s); + } + throw new IOException("Fail to load snapshot from " + path + ",content: " + s); + } +} diff --git a/jraft-example/src/main/resources/flexible_counter.proto b/jraft-example/src/main/resources/flexible_counter.proto new file mode 100644 index 000000000..4effe6231 --- /dev/null +++ b/jraft-example/src/main/resources/flexible_counter.proto @@ -0,0 +1,22 @@ +syntax = "proto2"; + +package jraft; + +option java_package = "com.alipay.sofa.jraft.example.flexibleRaft.rpc"; +option java_outer_classname = "FlexibleRaftOutter"; + + +message FlexibleGetValueRequest { + repeated bool readOnlySafe = 1; +} + +message FlexibleIncrementAndGetRequest { + required int64 delta = 1; +} + +message FlexibleValueResponse { + required int64 value = 1; + required bool success = 2; + optional string redirect = 3; + optional string errorMsg = 4; +} diff --git a/jraft-extension/bdb-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/impl/BDBLogStorage.java b/jraft-extension/bdb-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/impl/BDBLogStorage.java index 95ef83757..5fd425eb1 100644 --- a/jraft-extension/bdb-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/impl/BDBLogStorage.java +++ b/jraft-extension/bdb-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/impl/BDBLogStorage.java @@ -20,10 +20,12 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.util.ThreadPoolsFactory; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -159,9 +161,19 @@ private void load(final ConfigurationManager confManager) { if (entry.getType() == EntryType.ENTRY_TYPE_CONFIGURATION) { final ConfigurationEntry confEntry = new ConfigurationEntry(); confEntry.setId(new LogId(entry.getId().getIndex(), entry.getId().getTerm())); - confEntry.setConf(new Configuration(entry.getPeers(), entry.getLearners())); + Quorum quorum = new Quorum(entry.getQuorum().getW(), entry.getQuorum().getR()); + Configuration conf = new Configuration(entry.getPeers(), entry.getLearners(), quorum, + entry.getWriteFactor(), entry.getReadFactor(), entry.getEnableFlexible()); + confEntry.setConf(conf); + Quorum oldQuorum = null; + if (Objects.nonNull(entry.getOldQuorum())) { + oldQuorum = new Quorum(entry.getOldQuorum().getW(), entry.getOldQuorum().getR()); + } if (entry.getOldPeers() != null) { - confEntry.setOldConf(new Configuration(entry.getOldPeers(), entry.getOldLearners())); + Configuration oldConf = new Configuration(entry.getOldPeers(), entry.getOldLearners(), + oldQuorum, entry.getOldWriteFactor(), entry.getOldReadFactor(), + entry.getEnableFlexible()); + confEntry.setOldConf(oldConf); } if (confManager != null) { confManager.add(confEntry); diff --git a/jraft-extension/java-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/LogitLogStorage.java b/jraft-extension/java-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/LogitLogStorage.java index 6000d4de6..aa1c3d2ce 100644 --- a/jraft-extension/java-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/LogitLogStorage.java +++ b/jraft-extension/java-log-storage-impl/src/main/java/com/alipay/sofa/jraft/storage/LogitLogStorage.java @@ -20,10 +20,12 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.alipay.sofa.jraft.Quorum; import com.alipay.sofa.jraft.util.ThreadPoolsFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -272,10 +274,19 @@ public void loadConfiguration() { while ((entry = confIterator.next()) != null) { if (entry.getType() == EntryType.ENTRY_TYPE_CONFIGURATION) { final ConfigurationEntry confEntry = new ConfigurationEntry(); + Quorum quorum = new Quorum(entry.getQuorum().getW(), entry.getQuorum().getR()); + Configuration newConf = new Configuration(entry.getPeers(), entry.getLearners(), quorum, + entry.getWriteFactor(), entry.getReadFactor(), entry.getEnableFlexible()); + confEntry.setConf(newConf); + Quorum oldQuorum = null; confEntry.setId(new LogId(entry.getId().getIndex(), entry.getId().getTerm())); - confEntry.setConf(new Configuration(entry.getPeers(), entry.getLearners())); + if (Objects.nonNull(entry.getOldQuorum())) { + oldQuorum = new Quorum(entry.getOldQuorum().getW(), entry.getOldQuorum().getR()); + } if (entry.getOldPeers() != null) { - confEntry.setOldConf(new Configuration(entry.getOldPeers(), entry.getOldLearners())); + Configuration oldConf = new Configuration(entry.getOldPeers(), entry.getOldLearners(), oldQuorum, + entry.getOldWriteFactor(), entry.getOldReadFactor(), entry.getEnableFlexible()); + confEntry.setOldConf(oldConf); } if (this.configurationManager != null) { this.configurationManager.add(confEntry); diff --git a/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java b/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java index 3f7b726d8..b2aa708ff 100644 --- a/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java +++ b/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/core/NodeTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import com.alipay.sofa.jraft.entity.BallotFactory; import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.AfterClass; @@ -1946,7 +1947,9 @@ public void testSetPeer1() throws Exception { final List peers = new ArrayList<>(); peers.add(bootPeer); // reset peers from empty - assertTrue(nodes.get(0).resetPeers(new Configuration(peers)).isOk()); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(conf.size())); + assertTrue(nodes.get(0).resetPeers(conf).isOk()); cluster.waitLeader(); assertNotNull(cluster.getLeader()); @@ -2002,10 +2005,14 @@ public void testSetPeer2() throws Exception { newPeers.add(new PeerId(leaderAddr, 0)); // new peers equal to current conf - assertTrue(leader.resetPeers(new Configuration(peers)).isOk()); + Configuration conf = new Configuration(peers); + conf.setQuorum(BallotFactory.buildMajorityQuorum(conf.size())); + assertTrue(leader.resetPeers(conf).isOk()); // set peer when quorum die LOG.warn("Set peers to {}", leaderAddr); - assertTrue(leader.resetPeers(new Configuration(newPeers)).isOk()); + Configuration newConf = new Configuration(newPeers); + newConf.setQuorum(BallotFactory.buildMajorityQuorum(newConf.size())); + assertTrue(leader.resetPeers(newConf).isOk()); cluster.waitLeader(); leader = cluster.getLeader(); diff --git a/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/storage/BaseLogStorageTest.java b/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/storage/BaseLogStorageTest.java index 3e5ea8a7f..ffccd9586 100644 --- a/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/storage/BaseLogStorageTest.java +++ b/jraft-extension/java-log-storage-impl/src/test/java/com/alipay/sofa/jraft/storage/BaseLogStorageTest.java @@ -31,6 +31,7 @@ import com.alipay.sofa.jraft.entity.EnumOutter; import com.alipay.sofa.jraft.entity.LogEntry; import com.alipay.sofa.jraft.entity.LogId; +import com.alipay.sofa.jraft.entity.codec.v2.LogOutter; import com.alipay.sofa.jraft.option.LogStorageOptions; import com.alipay.sofa.jraft.test.TestUtils; @@ -164,11 +165,13 @@ public void testLoadWithConfigManager() { final LogEntry confEntry1 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry1.setId(new LogId(99, 1)); confEntry1.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082").listPeers()); - + confEntry1.setEnableFlexible(false); + confEntry1.setQuorum(LogOutter.Quorum.newBuilder().setR(2).setW(2).build()); final LogEntry confEntry2 = new LogEntry(EnumOutter.EntryType.ENTRY_TYPE_CONFIGURATION); confEntry2.setId(new LogId(100, 2)); confEntry2.setPeers(JRaftUtils.getConfiguration("localhost:8081,localhost:8082,localhost:8083").listPeers()); - + confEntry2.setEnableFlexible(false); + confEntry2.setQuorum(LogOutter.Quorum.newBuilder().setR(2).setW(2).build()); assertTrue(this.logStorage.appendEntry(confEntry1)); assertEquals(1, this.logStorage.appendEntries(Arrays.asList(confEntry2))); @@ -178,12 +181,15 @@ public void testLoadWithConfigManager() { ConfigurationEntry conf = this.confManager.getLastConfiguration(); assertNotNull(conf); + System.out.println("conf:" + conf); assertFalse(conf.isEmpty()); - assertEquals("localhost:8081,localhost:8082,localhost:8083", conf.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,localhost:8083,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", + conf.getConf().toString()); conf = this.confManager.get(99); assertNotNull(conf); assertFalse(conf.isEmpty()); - assertEquals("localhost:8081,localhost:8082", conf.getConf().toString()); + assertEquals("localhost:8081,localhost:8082,isEnableFlexible:false,quorum:Quorum{w=2, r=2}", conf.getConf() + .toString()); } @Test diff --git a/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/MarshallerHelper.java b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/MarshallerHelper.java index 4d0ba8033..7fde27bdc 100644 --- a/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/MarshallerHelper.java +++ b/jraft-extension/rpc-grpc-impl/src/main/java/com/alipay/sofa/jraft/rpc/impl/MarshallerHelper.java @@ -60,6 +60,8 @@ public class MarshallerHelper { CliRequests.LearnersOpResponse.getDefaultInstance()); messages.put(CliRequests.ResetLearnersRequest.class.getName(), CliRequests.LearnersOpResponse.getDefaultInstance()); + messages.put(CliRequests.ResetFactorRequest.class.getName(), + CliRequests.ResetFactorResponse.getDefaultInstance()); } public static Message findRespInstance(final String name) {