mirror of https://github.com/tikv/client-rust.git
260 lines
13 KiB
Protocol Buffer
260 lines
13 KiB
Protocol Buffer
syntax = "proto3";
|
|
package tikvpb;
|
|
|
|
import "coprocessor.proto";
|
|
import "kvrpcpb.proto";
|
|
import "mpp.proto";
|
|
import "raft_serverpb.proto";
|
|
import "disaggregated.proto";
|
|
|
|
import "gogoproto/gogo.proto";
|
|
import "rustproto.proto";
|
|
|
|
option (gogoproto.sizer_all) = true;
|
|
option (gogoproto.marshaler_all) = true;
|
|
option (gogoproto.unmarshaler_all) = true;
|
|
option (rustproto.lite_runtime_all) = true;
|
|
|
|
option java_package = "org.tikv.kvproto";
|
|
|
|
// Key/value store API for TiKV.
|
|
service Tikv {
|
|
// Commands using a transactional interface.
|
|
rpc KvGet(kvrpcpb.GetRequest) returns (kvrpcpb.GetResponse) {}
|
|
rpc KvScan(kvrpcpb.ScanRequest) returns (kvrpcpb.ScanResponse) {}
|
|
rpc KvPrewrite(kvrpcpb.PrewriteRequest) returns (kvrpcpb.PrewriteResponse) {}
|
|
rpc KvPessimisticLock(kvrpcpb.PessimisticLockRequest) returns (kvrpcpb.PessimisticLockResponse) {}
|
|
rpc KVPessimisticRollback(kvrpcpb.PessimisticRollbackRequest) returns (kvrpcpb.PessimisticRollbackResponse) {}
|
|
rpc KvTxnHeartBeat(kvrpcpb.TxnHeartBeatRequest) returns (kvrpcpb.TxnHeartBeatResponse) {}
|
|
rpc KvCheckTxnStatus(kvrpcpb.CheckTxnStatusRequest) returns (kvrpcpb.CheckTxnStatusResponse) {}
|
|
rpc KvCheckSecondaryLocks(kvrpcpb.CheckSecondaryLocksRequest) returns (kvrpcpb.CheckSecondaryLocksResponse) {}
|
|
rpc KvCommit(kvrpcpb.CommitRequest) returns (kvrpcpb.CommitResponse) {}
|
|
rpc KvImport(kvrpcpb.ImportRequest) returns (kvrpcpb.ImportResponse) {}
|
|
rpc KvCleanup(kvrpcpb.CleanupRequest) returns (kvrpcpb.CleanupResponse) {}
|
|
rpc KvBatchGet(kvrpcpb.BatchGetRequest) returns (kvrpcpb.BatchGetResponse) {}
|
|
rpc KvBatchRollback(kvrpcpb.BatchRollbackRequest) returns (kvrpcpb.BatchRollbackResponse) {}
|
|
rpc KvScanLock(kvrpcpb.ScanLockRequest) returns (kvrpcpb.ScanLockResponse) {}
|
|
rpc KvResolveLock(kvrpcpb.ResolveLockRequest) returns (kvrpcpb.ResolveLockResponse) {}
|
|
rpc KvGC(kvrpcpb.GCRequest) returns (kvrpcpb.GCResponse) {}
|
|
rpc KvDeleteRange(kvrpcpb.DeleteRangeRequest) returns (kvrpcpb.DeleteRangeResponse) {}
|
|
rpc KvPrepareFlashbackToVersion(kvrpcpb.PrepareFlashbackToVersionRequest) returns (kvrpcpb.PrepareFlashbackToVersionResponse) {}
|
|
rpc KvFlashbackToVersion(kvrpcpb.FlashbackToVersionRequest) returns (kvrpcpb.FlashbackToVersionResponse) {}
|
|
|
|
// Raw commands; no transaction support.
|
|
rpc RawGet(kvrpcpb.RawGetRequest) returns (kvrpcpb.RawGetResponse) {}
|
|
rpc RawBatchGet(kvrpcpb.RawBatchGetRequest) returns (kvrpcpb.RawBatchGetResponse) {}
|
|
rpc RawPut(kvrpcpb.RawPutRequest) returns (kvrpcpb.RawPutResponse) {}
|
|
rpc RawBatchPut(kvrpcpb.RawBatchPutRequest) returns (kvrpcpb.RawBatchPutResponse) {}
|
|
rpc RawDelete(kvrpcpb.RawDeleteRequest) returns (kvrpcpb.RawDeleteResponse) {}
|
|
rpc RawBatchDelete(kvrpcpb.RawBatchDeleteRequest) returns (kvrpcpb.RawBatchDeleteResponse) {}
|
|
rpc RawScan(kvrpcpb.RawScanRequest) returns (kvrpcpb.RawScanResponse) {}
|
|
rpc RawDeleteRange(kvrpcpb.RawDeleteRangeRequest) returns (kvrpcpb.RawDeleteRangeResponse) {}
|
|
rpc RawBatchScan(kvrpcpb.RawBatchScanRequest) returns (kvrpcpb.RawBatchScanResponse) {}
|
|
// Get TTL of the key. Returns 0 if TTL is not set for the key.
|
|
rpc RawGetKeyTTL(kvrpcpb.RawGetKeyTTLRequest) returns (kvrpcpb.RawGetKeyTTLResponse) {}
|
|
// Compare if the value in database equals to `RawCASRequest.previous_value` before putting the new value. If not, this request will have no effect and the value in the database will be returned.
|
|
rpc RawCompareAndSwap(kvrpcpb.RawCASRequest) returns (kvrpcpb.RawCASResponse) {}
|
|
rpc RawChecksum(kvrpcpb.RawChecksumRequest) returns (kvrpcpb.RawChecksumResponse) {}
|
|
|
|
// Store commands (sent to a each TiKV node in a cluster, rather than a certain region).
|
|
rpc UnsafeDestroyRange(kvrpcpb.UnsafeDestroyRangeRequest) returns (kvrpcpb.UnsafeDestroyRangeResponse) {}
|
|
rpc RegisterLockObserver(kvrpcpb.RegisterLockObserverRequest) returns (kvrpcpb.RegisterLockObserverResponse) {}
|
|
rpc CheckLockObserver(kvrpcpb.CheckLockObserverRequest) returns (kvrpcpb.CheckLockObserverResponse) {}
|
|
rpc RemoveLockObserver(kvrpcpb.RemoveLockObserverRequest) returns (kvrpcpb.RemoveLockObserverResponse) {}
|
|
rpc PhysicalScanLock(kvrpcpb.PhysicalScanLockRequest) returns (kvrpcpb.PhysicalScanLockResponse) {}
|
|
|
|
// Commands for executing SQL in the TiKV coprocessor (i.e., 'pushed down' to TiKV rather than
|
|
// executed in TiDB).
|
|
rpc Coprocessor(coprocessor.Request) returns (coprocessor.Response) {}
|
|
rpc CoprocessorStream(coprocessor.Request) returns (stream coprocessor.Response) {}
|
|
rpc BatchCoprocessor(coprocessor.BatchRequest) returns (stream coprocessor.BatchResponse) {}
|
|
|
|
// Command for executing custom user requests in TiKV coprocessor_v2.
|
|
rpc RawCoprocessor(kvrpcpb.RawCoprocessorRequest) returns (kvrpcpb.RawCoprocessorResponse) {}
|
|
|
|
// Raft commands (sent between TiKV nodes).
|
|
rpc Raft(stream raft_serverpb.RaftMessage) returns (raft_serverpb.Done) {}
|
|
rpc BatchRaft(stream BatchRaftMessage) returns (raft_serverpb.Done) {}
|
|
rpc Snapshot(stream raft_serverpb.SnapshotChunk) returns (raft_serverpb.Done) {}
|
|
rpc TabletSnapshot(stream raft_serverpb.TabletSnapshotRequest) returns (stream raft_serverpb.TabletSnapshotResponse) {}
|
|
|
|
// Sent from PD or TiDB to a TiKV node.
|
|
rpc SplitRegion (kvrpcpb.SplitRegionRequest) returns (kvrpcpb.SplitRegionResponse) {}
|
|
// Sent from TiFlash or TiKV to a TiKV node.
|
|
rpc ReadIndex(kvrpcpb.ReadIndexRequest) returns (kvrpcpb.ReadIndexResponse) {}
|
|
|
|
// Commands for debugging transactions.
|
|
rpc MvccGetByKey(kvrpcpb.MvccGetByKeyRequest) returns (kvrpcpb.MvccGetByKeyResponse) {}
|
|
rpc MvccGetByStartTs(kvrpcpb.MvccGetByStartTsRequest) returns (kvrpcpb.MvccGetByStartTsResponse) {}
|
|
|
|
// Batched commands.
|
|
rpc BatchCommands(stream BatchCommandsRequest) returns (stream BatchCommandsResponse) {}
|
|
|
|
// These are for mpp execution.
|
|
rpc DispatchMPPTask(mpp.DispatchTaskRequest) returns (mpp.DispatchTaskResponse) {}
|
|
rpc CancelMPPTask(mpp.CancelTaskRequest) returns (mpp.CancelTaskResponse) {}
|
|
rpc EstablishMPPConnection(mpp.EstablishMPPConnectionRequest) returns (stream mpp.MPPDataPacket) {}
|
|
rpc IsAlive(mpp.IsAliveRequest) returns (mpp.IsAliveResponse) {}
|
|
rpc ReportMPPTaskStatus(mpp.ReportTaskStatusRequest) returns (mpp.ReportTaskStatusResponse) {}
|
|
|
|
/// CheckLeader sends all information (includes region term and epoch) to other stores.
|
|
/// Once a store receives a request, it checks term and epoch for each region, and sends the regions whose
|
|
/// term and epoch match with local information in the store.
|
|
/// After the client collected all responses from all stores, it checks if got a quorum of responses from
|
|
/// other stores for every region, and decides to advance resolved ts from these regions.
|
|
rpc CheckLeader(kvrpcpb.CheckLeaderRequest) returns (kvrpcpb.CheckLeaderResponse);
|
|
|
|
/// Get the minimal `safe_ts` from regions at the store
|
|
rpc GetStoreSafeTS(kvrpcpb.StoreSafeTSRequest) returns (kvrpcpb.StoreSafeTSResponse);
|
|
|
|
/// Get the information about lock waiting from TiKV.
|
|
rpc GetLockWaitInfo(kvrpcpb.GetLockWaitInfoRequest) returns (kvrpcpb.GetLockWaitInfoResponse);
|
|
|
|
/// Compact a specified key range. This request is not restricted to raft leaders and will not be replicated.
|
|
/// It only compacts data on this node.
|
|
/// TODO: Currently this RPC is designed to be only compatible with TiFlash.
|
|
/// Shall be move out in https://github.com/pingcap/kvproto/issues/912
|
|
rpc Compact(kvrpcpb.CompactRequest) returns (kvrpcpb.CompactResponse);
|
|
/// Get the information about history lock waiting from TiKV.
|
|
rpc GetLockWaitHistory(kvrpcpb.GetLockWaitHistoryRequest) returns (kvrpcpb.GetLockWaitHistoryResponse);
|
|
|
|
/// Get system table from TiFlash
|
|
rpc GetTiFlashSystemTable(kvrpcpb.TiFlashSystemTableRequest) returns (kvrpcpb.TiFlashSystemTableResponse) {}
|
|
|
|
// These are for TiFlash disaggregated architecture
|
|
/// Try to lock a S3 object, atomically
|
|
rpc tryAddLock(disaggregated.TryAddLockRequest) returns (disaggregated.TryAddLockResponse) {};
|
|
/// Try to delete a S3 object, atomically
|
|
rpc tryMarkDelete(disaggregated.TryMarkDeleteRequest) returns (disaggregated.TryMarkDeleteResponse) {};
|
|
/// Build the disaggregated task on TiFlash write node
|
|
rpc EstablishDisaggTask(disaggregated.EstablishDisaggTaskRequest) returns (disaggregated.EstablishDisaggTaskResponse) {}
|
|
/// Cancel the disaggregated task on TiFlash write node
|
|
rpc CancelDisaggTask(disaggregated.CancelDisaggTaskRequest) returns (disaggregated.CancelDisaggTaskResponse) {}
|
|
/// Exchange page data between TiFlash write node and compute node
|
|
rpc FetchDisaggPages(disaggregated.FetchDisaggPagesRequest) returns (stream disaggregated.PagesPacket) {}
|
|
/// Compute node get configuration from Write node
|
|
rpc GetDisaggConfig(disaggregated.GetDisaggConfigRequest) returns (disaggregated.GetDisaggConfigResponse) {}
|
|
}
|
|
|
|
message BatchCommandsRequest {
|
|
repeated Request requests = 1;
|
|
repeated uint64 request_ids = 2;
|
|
|
|
message Request {
|
|
oneof cmd {
|
|
kvrpcpb.GetRequest Get = 1;
|
|
kvrpcpb.ScanRequest Scan = 2;
|
|
kvrpcpb.PrewriteRequest Prewrite = 3;
|
|
kvrpcpb.CommitRequest Commit = 4;
|
|
kvrpcpb.ImportRequest Import = 5;
|
|
kvrpcpb.CleanupRequest Cleanup = 6;
|
|
kvrpcpb.BatchGetRequest BatchGet = 7;
|
|
kvrpcpb.BatchRollbackRequest BatchRollback = 8;
|
|
kvrpcpb.ScanLockRequest ScanLock = 9;
|
|
kvrpcpb.ResolveLockRequest ResolveLock = 10;
|
|
kvrpcpb.GCRequest GC = 11;
|
|
kvrpcpb.DeleteRangeRequest DeleteRange = 12;
|
|
|
|
kvrpcpb.RawGetRequest RawGet = 13;
|
|
kvrpcpb.RawBatchGetRequest RawBatchGet = 14;
|
|
kvrpcpb.RawPutRequest RawPut = 15;
|
|
kvrpcpb.RawBatchPutRequest RawBatchPut = 16;
|
|
kvrpcpb.RawDeleteRequest RawDelete = 17;
|
|
kvrpcpb.RawBatchDeleteRequest RawBatchDelete = 18;
|
|
kvrpcpb.RawScanRequest RawScan = 19;
|
|
kvrpcpb.RawDeleteRangeRequest RawDeleteRange = 20;
|
|
kvrpcpb.RawBatchScanRequest RawBatchScan = 21;
|
|
|
|
coprocessor.Request Coprocessor = 22;
|
|
kvrpcpb.PessimisticLockRequest PessimisticLock = 23;
|
|
kvrpcpb.PessimisticRollbackRequest PessimisticRollback = 24;
|
|
|
|
kvrpcpb.CheckTxnStatusRequest CheckTxnStatus = 25;
|
|
kvrpcpb.TxnHeartBeatRequest TxnHeartBeat = 26;
|
|
|
|
kvrpcpb.CheckSecondaryLocksRequest CheckSecondaryLocks = 33;
|
|
|
|
kvrpcpb.RawCoprocessorRequest RawCoprocessor = 34;
|
|
|
|
kvrpcpb.FlashbackToVersionRequest FlashbackToVersion = 35;
|
|
kvrpcpb.PrepareFlashbackToVersionRequest PrepareFlashbackToVersion = 36;
|
|
|
|
// For some test cases.
|
|
BatchCommandsEmptyRequest Empty = 255;
|
|
}
|
|
|
|
// Reserved for deprecated VerKV
|
|
reserved 27 to 32;
|
|
}
|
|
}
|
|
|
|
message BatchCommandsResponse {
|
|
repeated Response responses = 1;
|
|
repeated uint64 request_ids = 2;
|
|
// 280 means TiKV gRPC cpu usage is 280%.
|
|
uint64 transport_layer_load = 3;
|
|
|
|
message Response {
|
|
oneof cmd {
|
|
kvrpcpb.GetResponse Get = 1;
|
|
kvrpcpb.ScanResponse Scan = 2;
|
|
kvrpcpb.PrewriteResponse Prewrite = 3;
|
|
kvrpcpb.CommitResponse Commit = 4;
|
|
kvrpcpb.ImportResponse Import = 5;
|
|
kvrpcpb.CleanupResponse Cleanup = 6;
|
|
kvrpcpb.BatchGetResponse BatchGet = 7;
|
|
kvrpcpb.BatchRollbackResponse BatchRollback = 8;
|
|
kvrpcpb.ScanLockResponse ScanLock = 9;
|
|
kvrpcpb.ResolveLockResponse ResolveLock = 10;
|
|
kvrpcpb.GCResponse GC = 11;
|
|
kvrpcpb.DeleteRangeResponse DeleteRange = 12;
|
|
|
|
kvrpcpb.RawGetResponse RawGet = 13;
|
|
kvrpcpb.RawBatchGetResponse RawBatchGet = 14;
|
|
kvrpcpb.RawPutResponse RawPut = 15;
|
|
kvrpcpb.RawBatchPutResponse RawBatchPut = 16;
|
|
kvrpcpb.RawDeleteResponse RawDelete = 17;
|
|
kvrpcpb.RawBatchDeleteResponse RawBatchDelete = 18;
|
|
kvrpcpb.RawScanResponse RawScan = 19;
|
|
kvrpcpb.RawDeleteRangeResponse RawDeleteRange = 20;
|
|
kvrpcpb.RawBatchScanResponse RawBatchScan = 21;
|
|
|
|
coprocessor.Response Coprocessor = 22;
|
|
kvrpcpb.PessimisticLockResponse PessimisticLock = 23;
|
|
kvrpcpb.PessimisticRollbackResponse PessimisticRollback = 24;
|
|
|
|
kvrpcpb.CheckTxnStatusResponse CheckTxnStatus = 25;
|
|
kvrpcpb.TxnHeartBeatResponse TxnHeartBeat = 26;
|
|
|
|
kvrpcpb.CheckSecondaryLocksResponse CheckSecondaryLocks = 33;
|
|
|
|
kvrpcpb.RawCoprocessorResponse RawCoprocessor = 34;
|
|
|
|
kvrpcpb.FlashbackToVersionResponse FlashbackToVersion = 35;
|
|
kvrpcpb.PrepareFlashbackToVersionResponse PrepareFlashbackToVersion = 36;
|
|
|
|
// For some test cases.
|
|
BatchCommandsEmptyResponse Empty = 255;
|
|
}
|
|
|
|
// Reserved for deprecated VerKV
|
|
reserved 27 to 32;
|
|
}
|
|
}
|
|
|
|
message BatchRaftMessage {
|
|
repeated raft_serverpb.RaftMessage msgs = 1;
|
|
}
|
|
|
|
message BatchCommandsEmptyRequest {
|
|
// ID of the test request.
|
|
uint64 test_id = 1;
|
|
// TiKV needs to delay at least such a time to response the client.
|
|
uint64 delay_time = 2;
|
|
}
|
|
|
|
message BatchCommandsEmptyResponse {
|
|
// ID of the test request.
|
|
uint64 test_id = 1;
|
|
}
|