blob: 439b17c40fa53524839aa9fcb6c14caff476e8a3 [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.hadoop";
option java_outer_classname = "HadoopProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
package ratis.hadoop;
import "Raft.proto";
service CombinedClientProtocolService {
rpc submitClientRequest(ratis.common.RaftClientRequestProto)
returns(ratis.common.RaftClientReplyProto);
rpc setConfiguration(ratis.common.SetConfigurationRequestProto)
returns(ratis.common.RaftClientReplyProto);
rpc groupManagement(ratis.common.GroupManagementRequestProto)
returns(ratis.common.RaftClientReplyProto);
rpc groupList(ratis.common.GroupListRequestProto)
returns(ratis.common.GroupListReplyProto);
rpc groupInfo(ratis.common.GroupInfoRequestProto)
returns(ratis.common.GroupInfoReplyProto);
rpc transferLeadership(ratis.common.TransferLeadershipRequestProto)
returns(ratis.common.RaftClientReplyProto);
rpc snapshotManagement(ratis.common.SnapshotManagementRequestProto)
returns(ratis.common.RaftClientReplyProto);
}
service RaftServerProtocolService {
rpc requestVote(ratis.common.RequestVoteRequestProto)
returns(ratis.common.RequestVoteReplyProto);
rpc appendEntries(ratis.common.AppendEntriesRequestProto)
returns(ratis.common.AppendEntriesReplyProto);
rpc installSnapshot(ratis.common.InstallSnapshotRequestProto)
returns(ratis.common.InstallSnapshotReplyProto);
}