blob: 17e7002881dbdb717a672bc314b574a652f482ac [file] [log] [blame]
/**
* 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.
*/
syntax = "proto3";
option java_package = "org.apache.ratis.proto";
option java_outer_classname = "RaftProtos";
option java_generate_equals_and_hash = true;
package ratis.common;
message RaftPeerProto {
bytes id = 1; // id of the peer
string address = 2; // e.g. address of the RPC server
uint32 priority = 3; // priority of the peer
string dataStreamAddress = 4; // address of the data stream server
string clientAddress = 5; // address of the client RPC server
string adminAddress = 6; // address of the admin RPC server
}
message RaftPeerIdProto {
bytes id = 1; // id of the peer
}
message RaftGroupIdProto {
bytes id = 1;
}
message RaftGroupProto {
RaftGroupIdProto groupId = 1;
repeated RaftPeerProto peers = 2;
}
message RaftGroupMemberIdProto {
bytes peerId = 1;
RaftGroupIdProto groupId = 2;
}
message RaftConfigurationProto {
repeated RaftPeerProto peers = 1; // the peers in the current or new conf
repeated RaftPeerProto oldPeers = 2; // the peers in the old conf
}
message StateMachineEntryProto {
/**
* StateMachine specific data which is not written to log.
* Unlike logEntryData, stateMachineData is managed and stored by the StateMachine but not the RaftLog.
*/
bytes stateMachineData = 1;
/**
* When stateMachineData is missing, it is the size of the serialized LogEntryProto along with stateMachineData.
* When stateMachineData is not missing, it must be set to zero.
*/
uint32 logEntryProtoSerializedSize = 2;
}
message StateMachineLogEntryProto {
// TODO: This is not super efficient if the SM itself uses PB to serialize its own data for a
/** RaftLog entry data */
bytes logData = 1;
/**
* StateMachine entry.
* StateMachine implementation may use this field to separate StateMachine specific data from the RaftLog data.
*/
StateMachineEntryProto stateMachineEntry = 2;
enum Type {
WRITE = 0;
DATASTREAM = 1;
}
Type type = 13;
// clientId and callId are used to rebuild the retry cache.
bytes clientId = 14;
uint64 callId = 15;
}
message MetadataProto {
uint64 commitIndex = 1;
}
message LogEntryProto {
uint64 term = 1;
uint64 index = 2;
oneof LogEntryBody {
StateMachineLogEntryProto stateMachineLogEntry = 3;
RaftConfigurationProto configurationEntry = 4;
MetadataProto metadataEntry = 5;
}
}
message TermIndexProto {
uint64 term = 1;
uint64 index = 2;
}
message RaftRpcRequestProto {
bytes requestorId = 1;
bytes replyId = 2;
RaftGroupIdProto raftGroupId = 3;
uint64 callId = 4;
bool toLeader = 5;
uint64 timeoutMs = 13;
RoutingTableProto routingTable = 14;
SlidingWindowEntry slidingWindowEntry = 15;
}
message SlidingWindowEntry {
uint64 seqNum = 1; // 0 for non-sliding-window requests; >= 1 for sliding-window requests
bool isFirst = 2; // Is this the first request of the sliding window?
}
message RaftRpcReplyProto {
bytes requestorId = 1;
bytes replyId = 2;
RaftGroupIdProto raftGroupId = 3;
uint64 callId = 4;
bool success = 15;
}
message FileChunkProto {
string filename = 1; // relative to root
uint64 totalSize = 2;
bytes fileDigest = 3;
uint32 chunkIndex = 4;
uint64 offset = 5;
bytes data = 6;
bool done = 7;
}
enum InstallSnapshotResult {
SUCCESS = 0;
NOT_LEADER = 1;
IN_PROGRESS = 2;
ALREADY_INSTALLED = 3;
CONF_MISMATCH = 4;
SNAPSHOT_INSTALLED = 5;
SNAPSHOT_UNAVAILABLE = 6;
}
message RequestVoteRequestProto {
RaftRpcRequestProto serverRequest = 1;
uint64 candidateTerm = 2;
TermIndexProto candidateLastEntry = 3;
bool preVote = 4;
}
message RequestVoteReplyProto {
RaftRpcReplyProto serverReply = 1;
uint64 term = 2;
bool shouldShutdown = 3;
}
message CommitInfoProto {
RaftPeerProto server = 1;
uint64 commitIndex = 2;
}
message AppendEntriesRequestProto {
RaftRpcRequestProto serverRequest = 1;
uint64 leaderTerm = 2;
TermIndexProto previousLog = 3;
repeated LogEntryProto entries = 4;
uint64 leaderCommit = 5;
bool initializing = 6;
repeated CommitInfoProto commitInfos = 15;
}
message AppendEntriesReplyProto {
enum AppendResult {
SUCCESS = 0;
NOT_LEADER = 1; // the requester's term is not large enough
INCONSISTENCY = 2; // gap between the local log and the entries or snapshot installation in progress or
// overlap between local snapshot and the entries
}
RaftRpcReplyProto serverReply = 1;
uint64 term = 2;
uint64 nextIndex = 3;
AppendResult result = 4;
uint64 followerCommit = 5;
uint64 matchIndex = 6;
bool isHearbeat = 7;
}
message InstallSnapshotRequestProto {
message SnapshotChunkProto {
string requestId = 1; // an identifier for chunked-requests.
uint32 requestIndex = 2; // the index for this request chunk. Starts from 0.
RaftConfigurationProto raftConfiguration = 3;
TermIndexProto termIndex = 4;
repeated FileChunkProto fileChunks = 5;
uint64 totalSize = 6;
bool done = 7; // whether this is the final chunk for the same req.
}
message NotificationProto {
TermIndexProto firstAvailableTermIndex = 1; // first available log index to notify Follower to install snapshot.
}
RaftRpcRequestProto serverRequest = 1;
uint64 leaderTerm = 2;
oneof InstallSnapshotRequestBody {
SnapshotChunkProto snapshotChunk = 3;
NotificationProto notification = 4;
}
LogEntryProto lastRaftConfigurationLogEntryProto = 5;
}
message InstallSnapshotReplyProto {
RaftRpcReplyProto serverReply = 1;
uint64 term = 2;
InstallSnapshotResult result = 3;
oneof InstallSnapshotReplyBody {
uint32 requestIndex = 4; // index of the snapshot chunk request.
uint64 snapshotIndex = 5; // index of snapshot installed after notification.
}
}
message ClientMessageEntryProto {
bytes content = 1;
}
enum ReplicationLevel {
/** Committed at the leader and replicated to the majority of peers. */
MAJORITY = 0;
/** Committed at the leader and replicated to all peers.
Note that ReplicationLevel.ALL implies ReplicationLevel.MAJORITY. */
ALL = 1;
/** Committed at majority peers.
Note that ReplicationLevel.MAJORITY_COMMITTED implies ReplicationLevel.MAJORITY. */
MAJORITY_COMMITTED = 2;
/** Committed at all peers.
Note that ReplicationLevel.ALL_COMMITTED implies ReplicationLevel.ALL
and ReplicationLevel.MAJORITY_COMMITTED */
ALL_COMMITTED = 3;
}
/** Role of raft peer */
enum RaftPeerRole {
LEADER = 0;
CANDIDATE = 1;
FOLLOWER = 2;
}
message WriteRequestTypeProto {
}
message MessageStreamRequestTypeProto {
uint64 streamId = 1; // the id of this stream
uint64 messageId = 2; // the message id within a particular stream.
bool endOfRequest = 3;// Is this the end-of-request?
}
message DataStreamRequestTypeProto {
}
message ForwardRequestTypeProto {
}
message ReadRequestTypeProto {
}
message StaleReadRequestTypeProto {
uint64 minIndex = 1;
}
message WatchRequestTypeProto {
uint64 index = 1;
ReplicationLevel replication = 2;
}
message RouteProto {
RaftPeerIdProto peerId = 1;
repeated RaftPeerIdProto successors = 2;
}
message RoutingTableProto {
repeated RouteProto routes = 1;
}
// normal client request
message RaftClientRequestProto {
RaftRpcRequestProto rpcRequest = 1;
ClientMessageEntryProto message = 2;
oneof Type {
WriteRequestTypeProto write = 3;
ReadRequestTypeProto read = 4;
StaleReadRequestTypeProto staleRead = 5;
WatchRequestTypeProto watch = 6;
MessageStreamRequestTypeProto messageStream = 7;
DataStreamRequestTypeProto dataStream = 8;
ForwardRequestTypeProto forward = 9;
}
}
message DataStreamPacketHeaderProto {
enum Type {
STREAM_HEADER = 0;
STREAM_DATA = 1;
}
enum Option {
SYNC = 0;
CLOSE = 1;
}
bytes clientId = 1;
Type type = 2;
uint64 streamId = 3;
uint64 streamOffset = 4;
uint64 dataLength = 5;
repeated Option options = 6;
}
message DataStreamRequestHeaderProto {
DataStreamPacketHeaderProto packetHeader = 1;
}
message DataStreamReplyHeaderProto {
DataStreamPacketHeaderProto packetHeader = 1;
uint64 bytesWritten = 2;
bool success = 3;
repeated CommitInfoProto commitInfos = 15;
}
message NotLeaderExceptionProto {
RaftPeerProto suggestedLeader = 1;
repeated RaftPeerProto peersInConf = 2;
}
message LeaderNotReadyExceptionProto {
RaftGroupMemberIdProto serverId = 1; // id of the leader
}
message NotReplicatedExceptionProto {
uint64 callId = 1;
ReplicationLevel replication = 2;
uint64 logIndex = 3;
}
message StateMachineExceptionProto {
string exceptionClassName = 1;
string errorMsg = 2;
bytes stacktrace = 3;
}
message AlreadyClosedExceptionProto {
string exceptionClassName = 1;
string errorMsg = 2;
bytes stacktrace = 3;
}
message ThrowableProto {
string className = 1;
string errorMessage = 2;
bytes stackTrace = 3;
bytes cause = 4;
}
message RaftClientReplyProto {
RaftRpcReplyProto rpcReply = 1;
ClientMessageEntryProto message = 2;
oneof ExceptionDetails {
NotLeaderExceptionProto notLeaderException = 3;
NotReplicatedExceptionProto notReplicatedException = 4;
StateMachineExceptionProto stateMachineException = 5;
LeaderNotReadyExceptionProto leaderNotReadyException = 6;
AlreadyClosedExceptionProto alreadyClosedException = 7;
ThrowableProto dataStreamException = 8;
ThrowableProto leaderSteppingDownException = 9;
ThrowableProto transferLeadershipException = 10;
}
uint64 logIndex = 14; // When the request is a write request and the reply is success, the log index of the transaction
repeated CommitInfoProto commitInfos = 15;
}
// setConfiguration request
message SetConfigurationRequestProto {
RaftRpcRequestProto rpcRequest = 1;
repeated RaftPeerProto peers = 2;
}
// transfer leadership request
message TransferLeadershipRequestProto {
RaftRpcRequestProto rpcRequest = 1;
RaftPeerProto newLeader = 2;
}
// snapshot request
message SnapshotManagementRequestProto {
RaftRpcRequestProto rpcRequest = 1;
oneof Op {
SnapshotCreateRequestProto create = 2;
}
}
message SnapshotCreateRequestProto {
}
message StartLeaderElectionRequestProto {
RaftRpcRequestProto serverRequest = 1;
TermIndexProto leaderLastEntry = 2;
}
message StartLeaderElectionReplyProto {
RaftRpcReplyProto serverReply = 1;
}
// A request to add a new group
message GroupAddRequestProto {
RaftGroupProto group = 1; // the group to be added.
}
message GroupRemoveRequestProto {
RaftGroupIdProto groupId = 1; // the group to be removed.
bool deleteDirectory = 2; // delete the directory for that group?
bool renameDirectory = 3; // rename the directory if not deleted.
}
message GroupManagementRequestProto {
RaftRpcRequestProto rpcRequest = 1;
oneof Op {
GroupAddRequestProto groupAdd = 2;
GroupRemoveRequestProto groupRemove = 3;
}
}
// server info requests
message GroupListRequestProto {
RaftRpcRequestProto rpcRequest = 1;
}
message GroupInfoRequestProto {
RaftRpcRequestProto rpcRequest = 1;
RaftGroupIdProto groupId = 2;
}
message ServerRpcProto {
RaftPeerProto id = 1;
uint64 lastRpcElapsedTimeMs = 2;
}
message LeaderInfoProto {
repeated ServerRpcProto followerInfo = 1;
uint64 term = 2;
}
message FollowerInfoProto {
ServerRpcProto leaderInfo = 1;
uint32 outstandingOp = 2;
}
message CandidateInfoProto {
uint64 lastLeaderElapsedTimeMs = 1;
}
message RoleInfoProto {
RaftPeerProto self = 1;
RaftPeerRole role = 2;
uint64 roleElapsedTimeMs = 3;
oneof PeerInfo {
LeaderInfoProto leaderInfo = 4;
FollowerInfoProto followerInfo = 5;
CandidateInfoProto candidateInfo = 6;
}
}
message GroupListReplyProto {
RaftRpcReplyProto rpcReply = 1;
repeated RaftGroupIdProto groupId = 2;
}
message GroupInfoReplyProto {
RaftRpcReplyProto rpcReply = 1;
RaftGroupProto group = 2;
RoleInfoProto role = 3;
bool isRaftStorageHealthy = 4;
repeated CommitInfoProto commitInfos = 5;
}