// Copyright 2016 The Cockroach Authors. // // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. // // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. syntax = "proto3"; package cockroach.server.serverpb; option go_package = "serverpb"; import "build/info.proto"; import "gossip/gossip.proto"; import "jobs/jobspb/jobs.proto"; import "roachpb/app_stats.proto"; import "roachpb/data.proto"; import "roachpb/index_usage_stats.proto"; import "roachpb/metadata.proto"; import "server/diagnostics/diagnosticspb/diagnostics.proto"; import "server/status/statuspb/status.proto"; import "sql/contentionpb/contention.proto"; import "sql/execinfrapb/api.proto"; import "storage/enginepb/engine.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/rocksdb.proto"; import "kv/kvserver/kvserverpb/lease_status.proto"; import "kv/kvserver/kvserverpb/state.proto"; import "kv/kvserver/liveness/livenesspb/liveness.proto"; import "util/log/logpb/log.proto"; import "util/unresolved_addr.proto"; import "util/hlc/timestamp.proto"; import "etcd/raft/v3/raftpb/raft.proto"; import "gogoproto/gogo.proto"; import "google/api/annotations.proto"; import "google/protobuf/timestamp.proto"; import "google/protobuf/duration.proto"; message CertificatesRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message CertificateDetails { // We use an enum to allow reporting of client certs and potential others (eg: // UI). enum CertificateType { CA = 0; NODE = 1; CLIENT_CA = 2; CLIENT = 3; UI_CA = 4; UI = 5; } message Fields { string issuer = 1; string subject = 2; int64 valid_from = 3; int64 valid_until = 4; repeated string addresses = 5; string signature_algorithm = 6; string public_key = 7; repeated string key_usage = 8; repeated string extended_key_usage = 9; } CertificateType type = 1; // "error_message" and "data" are mutually exclusive. string error_message = 2; // data is the raw file contents of the certificate. This means PEM-encoded // DER data. bytes data = 3; repeated Fields fields = 4 [ (gogoproto.nullable) = false ]; } message CertificatesResponse { repeated CertificateDetails certificates = 1 [ (gogoproto.nullable) = false ]; } // DetailsRequest requests a nodes details. // Note: this does *not* check readiness. Use the Health RPC for that purpose. message DetailsRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; reserved 2; } // SystemInfo contains information about the host system. message SystemInfo { // system_info is the output from `uname -a` string system_info = 1; // kernel_info is the output from `uname -r`. string kernel_info = 2; } message DetailsResponse { int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; util.UnresolvedAddr address = 2 [ (gogoproto.nullable) = false ]; build.Info build_info = 3 [ (gogoproto.nullable) = false ]; SystemInfo system_info = 4 [ (gogoproto.nullable) = false ]; util.UnresolvedAddr sql_address = 5 [ (gogoproto.nullable) = false, (gogoproto.customname) = "SQLAddress" ]; } // NodesRequest requests a copy of the node information as known to gossip // and the KV layer. // API: PUBLIC ALPHA message NodesRequest {} // NodesResponse describe the nodes in the cluster. // API: PUBLIC ALPHA message NodesResponse { // nodes carries the status payloads for all nodes in the cluster. // API: PUBLIC ALPHA repeated server.status.statuspb.NodeStatus nodes = 1 [ (gogoproto.nullable) = false ]; // liveness_by_node_id maps each node ID to a liveness status. map<int32, kv.kvserver.liveness.livenesspb.NodeLivenessStatus> liveness_by_node_id = 2 [ (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID", (gogoproto.nullable) = false, (gogoproto.customname) = "LivenessByNodeID" ]; } // NodesResponse describe the nodes in the cluster. // API: PUBLIC ALPHA message NodesResponseExternal { // nodes carries the status payloads for all nodes in the cluster. // API: PUBLIC ALPHA repeated NodeResponse nodes = 1 [ (gogoproto.nullable) = false ]; // liveness_by_node_id maps each node ID to a liveness status. map<int32, kv.kvserver.liveness.livenesspb.NodeLivenessStatus> liveness_by_node_id = 2 [ (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID", (gogoproto.nullable) = false, (gogoproto.customname) = "LivenessByNodeID" ]; } // Locality is an ordered set of key value Tiers that describe a node's // location. The tier keys should be the same across all nodes. message Locality { repeated Tier tiers = 1 [(gogoproto.nullable) = false]; } // Tier represents one level of the locality hierarchy. message Tier { // Key is the name of tier and should match all other nodes. string key = 1; // Value is node specific value corresponding to the key. string value = 2; } message Version { // The names "major" and "minor" are reserved in C in // some platforms (e.g. FreeBSD). int32 major_val = 1 [(gogoproto.customname) = "Major"]; int32 minor_val = 2 [(gogoproto.customname) = "Minor"]; // Note that patch is a placeholder and will always be zero. int32 patch = 3; // The internal version is used to introduce migrations during the development // cycle. They are subversions that are never the end versions of a release, // i.e. users of stable, public release will only use binaries with the // internal version set to 0. int32 internal = 4; } // NodeDescriptor holds details on node physical/network topology. message NodeDescriptor { int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; // address should only be visible by the Admin role. util.UnresolvedAddr address = 2 [(gogoproto.nullable) = false]; // attrs should only be visible by the Admin role. cockroach.roachpb.Attributes attrs = 3 [(gogoproto.nullable) = false]; Locality locality = 4 [(gogoproto.nullable) = false]; Version ServerVersion = 5 [(gogoproto.nullable) = false]; string build_tag = 6; int64 started_at = 7; // locality_address should only be visible by the Admin role. repeated cockroach.roachpb.LocalityAddress locality_address = 8 [(gogoproto.nullable) = false]; string cluster_name = 9; // The SQL address. If empty, indicates that the base address field // is also used to accept SQL connections. // sql_address should only be visible by the Admin role. util.UnresolvedAddr sql_address = 10 [(gogoproto.nullable) = false, (gogoproto.customname) = "SQLAddress"]; } // Percentiles contains a handful of hard-coded percentiles meant to summarize // a distribution. message Percentiles { double p10 = 1; double p25 = 2; double p50 = 3; double p75 = 4; double p90 = 5; double pMax = 6; } // StoreDescriptor holds store information including store attributes, node // descriptor and store capacity. message StoreDescriptor { int32 store_id = 1 [(gogoproto.customname) = "StoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID"]; roachpb.Attributes attrs = 2 [(gogoproto.nullable) = false]; NodeDescriptor node = 3 [(gogoproto.nullable) = false]; roachpb.StoreCapacity capacity = 4 [(gogoproto.nullable) = false]; roachpb.StoreProperties properties = 5 [(gogoproto.nullable) = false]; } // StoreStatus records the most recent values of metrics for a store. message StoreStatus { // desc is the store descriptor. StoreDescriptor desc = 1 [(gogoproto.nullable) = false]; // metrics contains the last sampled values for the node metrics. map<string, double> metrics = 2; } // NodeStatus records the most recent values of metrics for a node. // API: PUBLIC ALPHA message NodeResponse { // desc is the node descriptor. NodeDescriptor desc = 1 [(gogoproto.nullable) = false]; // build_info describes the `cockroach` executable file. // API: PUBLIC ALPHA build.Info build_info = 2 [(gogoproto.nullable) = false]; // started_at is the unix timestamp at which the node process was // last started. // API: PUBLIC ALPHA int64 started_at = 3; // updated_at is the unix timestamp at which the node status record // was last updated. // API: PUBLIC ALPHA int64 updated_at = 4; // metrics contains the last sampled values for the node metrics. map<string, double> metrics = 5; // store_statuses provides the store status payloads for all // the stores on that node. repeated StoreStatus store_statuses = 6 [(gogoproto.nullable) = false]; // args is the list of command-line arguments used to last start the node. // args should only be visible by the Admin role. repeated string args = 7; // env is the list of environment variables that influenced // the node's configuration. // env should only be visible by the Admin role. repeated string env = 8; // latencies is a map of nodeIDs to nanoseconds which is the latency // between this node and the other node. // // NOTE: this is deprecated and is only set if the min supported // cluster version is >= VersionRPCNetworkStats. map<int32, int64> latencies = 9 [ (gogoproto.nullable) = false, (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; message NetworkActivity { int64 incoming = 1; // in bytes int64 outgoing = 2; // in bytes int64 latency = 3; // in nanoseconds } // activity is a map of nodeIDs to network statistics from this node // to other nodes. map<int32, NetworkActivity> activity = 10 [ (gogoproto.nullable) = false, (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; // total_system_memory is the total RAM available to the system // (or, if detected, the memory available to the cgroup this process is in) // in bytes. // API: PUBLIC ALPHA int64 total_system_memory = 11; // num_cpus is the number of logical CPUs as reported by the operating system // on the host where the `cockroach` process is running. Note that // this does not report the number of CPUs actually used by `cockroach`; // this parameter is controlled separately. // API: PUBLIC ALPHA int32 num_cpus = 12; } // RegionsRequest requests all available regions. message RegionsRequest {} // RegionsResponse describes the available regions. message RegionsResponse { message Region { repeated string zones = 1; } map<string, Region> regions = 1; } message NodeRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } // RaftState gives internal details about a Raft group's state. // Closely mirrors the upstream definitions in github.com/etcd-io/etcd/raft. message RaftState { message Progress { uint64 match = 1; uint64 next = 2; string state = 3; bool paused = 4; uint64 pending_snapshot = 5; } uint64 replica_id = 1 [ (gogoproto.customname) = "ReplicaID" ]; raftpb.HardState hard_state = 2 [ (gogoproto.nullable) = false ]; // Lead is part of Raft's SoftState. uint64 lead = 3; // State is part of Raft's SoftState. // It's not an enum because this is primarily for ui consumption and there // are issues associated with them. string state = 4; uint64 applied = 5; map<uint64, Progress> progress = 6 [ (gogoproto.nullable) = false ]; uint64 lead_transferee = 7; } // RangeProblems describes issues reported by a range. For internal use only. message RangeProblems { bool unavailable = 1; bool leader_not_lease_holder = 2; bool no_raft_leader = 3; bool underreplicated = 4; bool overreplicated = 8; bool no_lease = 5; // Quiescent ranges do not tick by definition, but we track this in // two different ways and suspect that they're getting out of sync. // If the replica's quiescent flag doesn't agree with the store's // list of replicas that are ticking, warn about it. bool quiescent_equals_ticking = 6; // When the raft log is too large, it can be a symptom of other issues. bool raft_log_too_large = 7; } // RangeStatistics describes statistics reported by a range. For internal use // only. message RangeStatistics { // Queries per second served by this range. // // Note that queries per second will only be known by the leaseholder. // All other replicas will report it as 0. double queries_per_second = 1; // Writes per second served by this range. double writes_per_second = 2; } message PrettySpan { option (gogoproto.equal) = true; string start_key = 1; string end_key = 2; } message RangeInfo { PrettySpan span = 1 [ (gogoproto.nullable) = false ]; RaftState raft_state = 2 [ (gogoproto.nullable) = false ]; kv.kvserver.storagepb.RangeInfo state = 4 [ (gogoproto.nullable) = false ]; int32 source_node_id = 5 [ (gogoproto.customname) = "SourceNodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; int32 source_store_id = 6 [ (gogoproto.customname) = "SourceStoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" ]; string error_message = 7; repeated roachpb.Lease lease_history = 8 [ (gogoproto.nullable) = false ]; RangeProblems problems = 9 [ (gogoproto.nullable) = false ]; RangeStatistics stats = 10 [ (gogoproto.nullable) = false ]; reserved 11, 12; kv.kvserver.storagepb.LeaseStatus lease_status = 13 [ (gogoproto.nullable) = false ]; bool quiescent = 14; bool ticking = 15; int64 read_latches = 16; int64 write_latches = 17; int64 locks = 18; int64 locks_with_wait_queues = 19; int64 lock_wait_queue_waiters = 20; message LockInfo { string pretty_key = 1; bytes key = 2 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.Key" ]; bool held = 3; int64 waiters = 4; int64 waiting_readers = 5; int64 waiting_writers = 6; } repeated LockInfo top_k_locks_by_wait_queue_waiters = 21 [ (gogoproto.nullable) = false ]; } message RangesRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; repeated int64 range_ids = 2 [ (gogoproto.customname) = "RangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; } message RangesResponse { repeated RangeInfo ranges = 1 [ (gogoproto.nullable) = false ]; } message GossipRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message EngineStatsInfo { int32 store_id = 1 [ (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" ]; cockroach.storage.enginepb.TickersAndHistograms tickers_and_histograms = 2; cockroach.storage.enginepb.EngineType engine_type = 3; } message EngineStatsRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message EngineStatsResponse { repeated EngineStatsInfo stats = 1 [ (gogoproto.nullable) = false ]; } message TraceEvent { google.protobuf.Timestamp time = 1 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; string message = 2; } message AllocatorDryRun { int64 range_id = 1 [ (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated TraceEvent events = 2; } message AllocatorRangeRequest { int64 range_id = 1; } message AllocatorRangeResponse { // The NodeID of the store whose dry run is returned. Only the leaseholder // for a given range will do an allocator dry run for it. int64 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; AllocatorDryRun dry_run = 2; } message AllocatorRequest { string node_id = 1; repeated int64 range_ids = 2 [ (gogoproto.customname) = "RangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; } message AllocatorResponse { repeated AllocatorDryRun dry_runs = 1; } message JSONResponse { bytes data = 1; } message ResponseError { string error = 1; int32 code = 2; string message = 3; repeated string details = 4; } message LogsRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; string level = 2; string start_time = 3; string end_time = 4; string max = 5; string pattern = 6; // redact, if true, requests redaction of sensitive data away // from the retrieved log entries. // Only admin users can send a request with redact = false. bool redact = 7; reserved 8; } message LogEntriesResponse { repeated cockroach.util.log.Entry entries = 1 [ (gogoproto.nullable) = false ]; } message LogFilesListRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message LogFilesListResponse { // files is the list of log files under this node's configured logging // directories. Note that the response does not contain detail about // which directory contains which file. The location of each file // is known to the server based on its name and the logging configuration. repeated cockroach.util.log.FileInfo files = 1 [ (gogoproto.nullable) = false ]; } message LogFileRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; // file is the name of the log file to retrieve. Note that it // must not be prefixed by a directory name. The full path to // the file is computed by the server based on the base name // and the logging configuration. string file = 2; // redact, if true, requests redaction of sensitive data away // from the retrieved log entries. // Only admin users can send a request with redact = false. bool redact = 3; reserved 4; } enum StacksType { // GOROUTINE_STACKS corresponds to GOROUTINE_STACKS_DEBUG_2. GOROUTINE_STACKS = 0; reserved 1; GOROUTINE_STACKS_DEBUG_1 = 2; } message StacksRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; StacksType type = 2; } // Represents the type of file. // TODO(ridwanmsharif): Add support for log files. They're currently served // by an endpoint that parses the log messages, which is not what the // debug zip client wants. enum FileType { HEAP = 0; GOROUTINES = 1; } message File { string name = 1; int64 file_size = 2; // Contents may not be populated if only a list of Files are requested. bytes contents = 3; } message GetFilesRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; // If list_only is true then the contents of the files will not be populated // in the response. Only filenames and sizes will be returned. bool list_only = 2; FileType type = 5; // Each pattern given is matched with Files of the above type in the node // using filepath.Glob(). The patterns only match to filenames and so path // separators cannot be used. // Example: * will match all files of requested type. repeated string patterns = 6; } message GetFilesResponse { repeated File files = 1; } message ProfileRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; enum Type { HEAP = 0; CPU = 1; // with labels on by default GOROUTINE = 2; THREADCREATE = 3; ALLOCS = 4; BLOCK = 5; MUTEX = 6; } // The type of profile to retrieve. Type type = 5; int32 seconds = 6; // applies only to Type=CPU, defaults to 30 bool labels = 7; // applies only to Type=CPU, defaults to false } message MetricsRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message RaftRangeNode { int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; RangeInfo range = 2 [ (gogoproto.nullable) = false ]; } message RaftRangeError { string message = 1; } message RaftRangeStatus { int64 range_id = 1 [ (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated RaftRangeError errors = 2 [ (gogoproto.nullable) = false ]; repeated RaftRangeNode nodes = 3 [ (gogoproto.nullable) = false ]; } message RaftDebugRequest { repeated int64 range_ids = 1 [ (gogoproto.customname) = "RangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; } message RaftDebugResponse { map<int64, RaftRangeStatus> ranges = 1 [ (gogoproto.nullable) = false, (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated RaftRangeError errors = 2 [ (gogoproto.nullable) = false ]; } // TxnInfo represents an in flight user transaction on some Session. message TxnInfo { bytes id = 1 [ (gogoproto.customname) = "ID", (gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" ]; // The start timestamp of the transaction. google.protobuf.Timestamp start = 2 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; // txn_description is a text description of the underlying kv.Txn, intended // for troubleshooting purposes. string txn_description = 3; // num_statements_executed is the number of statements that were executed so // far on this transaction. int32 num_statements_executed = 4; // num_retries is the number of times that this transaction was retried. int32 num_retries = 5; // num_retries is the number of times that this transaction was automatically // retried by the SQL executor. int32 num_auto_retries = 6; // The deadline by which the transaction must be committed. google.protobuf.Timestamp deadline = 7 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; // implicit is true if this transaction was an implicit SQL transaction. bool implicit = 8; // Number of currently allocated bytes in the txn memory monitor. int64 alloc_bytes = 9; // High water mark of allocated bytes in the txn memory monitor. int64 max_alloc_bytes = 10; bool read_only = 11; bool is_historical = 12; string priority = 13; } // ActiveQuery represents a query in flight on some Session. message ActiveQuery { // ID of the query (uint128 presented as a hexadecimal string). string id = 1 [ (gogoproto.customname) = "ID" ]; // The UUID of the transaction this query is running in. bytes txn_id = 7 [ (gogoproto.customname) = "TxnID", (gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" ]; // SQL query string specified by the user. string sql = 2; // Start timestamp of this query. google.protobuf.Timestamp start = 3 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; // True if this query is distributed. bool is_distributed = 4; // Enum for phase of execution. enum Phase { PREPARING = 0; EXECUTING = 1; } // phase stores the current phase of execution for this query. Phase phase = 5; // progress is an estimate of the fraction of this query that has been // processed. float progress = 6; // The SQL statement fingerprint, compatible with StatementStatisticsKey. string sql_no_constants = 8; // A summarized version of the sql query. string sql_summary = 9; } // Request object for ListSessions and ListLocalSessions. message ListSessionsRequest { // Username of the user making this request. // The caller is responsible to normalize the username // (= case fold and perform unicode NFC normalization). string username = 1; } // Session represents one SQL session. message Session { // ID of node where this session exists. int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; // Username of the user for this session. string username = 2; // Connected client's IP address and port. string client_address = 3; // Application name specified by the client. string application_name = 4; // Queries in progress on this session. repeated ActiveQuery active_queries = 5 [ (gogoproto.nullable) = false ]; // Timestamp of session's start. google.protobuf.Timestamp start = 6 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; // This used to be active_txn_id until it was moved inside of active_txn. reserved 7; // SQL string of the last query executed on this session. string last_active_query = 8; // ID of the session (uint128 represented as raw bytes). bytes id = 9 [ (gogoproto.customname) = "ID" ]; // Number of currently allocated bytes in the session memory monitor. int64 alloc_bytes = 10; // High water mark of allocated bytes in the session memory monitor. int64 max_alloc_bytes = 11; // Information about the txn in progress on this session. Nil if the // session doesn't currently have a transaction. TxnInfo active_txn = 12; // The SQL statement fingerprint of the last query executed on this session, // compatible with StatementStatisticsKey. string last_active_query_no_constants = 13; } // An error wrapper object for ListSessionsResponse. message ListSessionsError { // ID of node that was being contacted when this error occurred int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; // Error message. string message = 2; } // Response object for ListSessions and ListLocalSessions. message ListSessionsResponse { // A list of sessions on this node or cluster. repeated Session sessions = 1 [ (gogoproto.nullable) = false ]; // Any errors that occurred during fan-out calls to other nodes. repeated ListSessionsError errors = 2 [ (gogoproto.nullable) = false ]; } // Request object for issuing a query cancel request. message CancelQueryRequest { // ID of gateway node for the query to be canceled. // // TODO(itsbilal): use [(gogoproto.customname) = "NodeID"] below. Need to // figure out how to teach grpc-gateway about custom names. // // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; // ID of query to be canceled (converted to string). string query_id = 2 [(gogoproto.customname) = "QueryID" ]; // Username of the user making this cancellation request. This may be omitted // if the user is the same as the one issuing the CancelQueryRequest. // The caller is responsible for case-folding and NFC normalization. string username = 3; } // Response returned by target query's gateway node. message CancelQueryResponse { // Whether the cancellation request succeeded and the query was canceled. bool canceled = 1; // Error message (accompanied with canceled = false). string error = 2; } message CancelSessionRequest { // TODO(abhimadan): use [(gogoproto.customname) = "NodeID"] below. Need to // figure out how to teach grpc-gateway about custom names. // // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; bytes session_id = 2 [(gogoproto.customname) = "SessionID"]; // Username of the user making this cancellation request. This may be omitted // if the user is the same as the one issuing the CancelSessionRequest. // The caller is responsible for case-folding and NFC normalization. string username = 3; } message CancelSessionResponse { bool canceled = 1; string error = 2; } // Request object for ListContentionEvents and ListLocalContentionEvents. message ListContentionEventsRequest {} // An error wrapper object for ListContentionEventsResponse and // ListDistSQLFlowsResponse. Similar to the Statements endpoint, when // implemented on a tenant, the `node_id` field refers to the instanceIDs that // identify individual tenant pods. message ListActivityError { // ID of node that was being contacted when this error occurred. int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; // Error message. string message = 2; } // Response object for ListContentionEvents and ListLocalContentionEvents. message ListContentionEventsResponse { // All available contention information on this node or cluster. cockroach.sql.contentionpb.SerializedRegistry events = 1 [ (gogoproto.nullable) = false ]; // Any errors that occurred during fan-out calls to other nodes. repeated ListActivityError errors = 2 [ (gogoproto.nullable) = false ]; } // Request object for ListDistSQLFlows and ListLocalDistSQLFlows. message ListDistSQLFlowsRequest {} // DistSQLRemoteFlows describes all DistSQL remote flows that are part of the // same physical plan. The gateway node that initiated the execution of the plan // isn't included. message DistSQLRemoteFlows { enum Status { RUNNING = 0; QUEUED = 1; } // Info contains an information about a single DistSQL remote flow. message Info { // NodeID is the node on which this remote flow is either running or queued. int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; // Timestamp must be in the UTC timezone. google.protobuf.Timestamp timestamp = 2 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; // Status is the current status of this remote flow. Status status = 3; // Stmt is the SQL statement for which this flow is executing. string stmt = 4; } // FlowID is the unique identifier of the physical plan shared by all remote // flows. bytes flow_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "FlowID", (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb.FlowID"]; // Infos contains the information about all remote flows that are part of this // DistSQL physical plan and that haven't finished yet. // // The slice doesn't contain duplicate NodeIDs and is sorted by NodeID in ASC // order. repeated Info infos = 2 [(gogoproto.nullable) = false]; } // Response object for ListDistSQLFlows and ListLocalDistSQLFlows. message ListDistSQLFlowsResponse { // Flows are ordered by FlowID lexicographically and do not contain // duplicates. repeated DistSQLRemoteFlows flows = 1 [ (gogoproto.nullable) = false ]; // Any errors that occurred during fan-out calls to other nodes. repeated ListActivityError errors = 2 [ (gogoproto.nullable) = false ]; } message SpanStatsRequest { string node_id = 1 [ (gogoproto.customname) = "NodeID" ]; bytes start_key = 2 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RKey" ]; bytes end_key = 3 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RKey" ]; } message SpanStatsResponse { int32 range_count = 2; uint64 approximate_disk_bytes = 3; cockroach.storage.enginepb.MVCCStats total_stats = 1 [ (gogoproto.nullable) = false ]; } message ProblemRangesRequest { // If left empty, problem ranges for all nodes/stores will be returned. string node_id = 1 [ (gogoproto.customname) = "NodeID" ]; } message ProblemRangesResponse { message NodeProblems { string error_message = 1; repeated int64 unavailable_range_ids = 2 [ (gogoproto.customname) = "UnavailableRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 raft_leader_not_lease_holder_range_ids = 3 [ (gogoproto.customname) = "RaftLeaderNotLeaseHolderRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 no_raft_leader_range_ids = 4 [ (gogoproto.customname) = "NoRaftLeaderRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 no_lease_range_ids = 5 [ (gogoproto.customname) = "NoLeaseRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 underreplicated_range_ids = 6 [ (gogoproto.customname) = "UnderreplicatedRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 overreplicated_range_ids = 9 [ (gogoproto.customname) = "OverreplicatedRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 quiescent_equals_ticking_range_ids = 7 [ (gogoproto.customname) = "QuiescentEqualsTickingRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; repeated int64 raft_log_too_large_range_ids = 8 [ (gogoproto.customname) = "RaftLogTooLargeRangeIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; } reserved 1 to 7; // NodeID is the node that submitted all the requests. int32 node_id = 8 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; map<int32, NodeProblems> problems_by_node_id = 9 [ (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID", (gogoproto.customname) = "ProblemsByNodeID", (gogoproto.nullable) = false ]; } // HotRangesRequest queries one or more cluster nodes for a list // of ranges currently considered “hot” by the node(s). // API: PUBLIC ALPHA message HotRangesRequest { // NodeID indicates which node to query for a hot range report. // It is possible to populate any node ID; if the node receiving // the request is not the target node, it will forward the // request to the target node. // // If left empty, the request is forwarded to every node // in the cluster. // API: PUBLIC ALPHA string node_id = 1 [(gogoproto.customname) = "NodeID"]; } // HotRangesResponse is the payload produced in response // to a HotRangesRequest. // API: PUBLIC ALPHA message HotRangesResponse { // NodeID is the node that received the HotRangesRequest and // forwarded requests to the selected target node(s). // API: PUBLIC ALPHA int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; // HotRange is a hot range report for a single store on one of the // target node(s) selected in a HotRangesRequest. // API: PUBLIC ALPHA message HotRange { // Desc is the descriptor of the range for which the report // was produced. // // TODO(knz): This field should be removed. // See: https://github.com/cockroachdb/cockroach/issues/53212 cockroach.roachpb.RangeDescriptor desc = 1 [(gogoproto.nullable) = false]; // QueriesPerSecond is the recent number of queries per second // on this range. // API: PUBLIC ALPHA double queries_per_second = 2; } // StoreResponse contains the part of a hot ranges report that // pertains to a single store on a target node. // API: PUBLIC ALPHA message StoreResponse { // StoreID identifies the store for which the report was // produced. // API: PUBLIC ALPHA int32 store_id = 1 [ (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" ]; // HotRanges is the hot ranges report for this store // on the target node. // API: PUBLIC ALPHA repeated HotRange hot_ranges = 2 [(gogoproto.nullable) = false]; } // NodeResponse is a hot range report for a single target node. // API: PUBLIC ALPHA message NodeResponse { // ErrorMessage is set to a non-empty string if this target // node was unable to produce a hot range report. // // The contents of this string indicates the cause of the failure. // API: PUBLIC ALPHA string error_message = 1; // Stores contains the hot ranges report if no error was encountered. // There is one part to the report for each store in the // target node. // API: PUBLIC ALPHA repeated StoreResponse stores = 2; } // HotRangesByNodeID contains a hot range report for each selected // target node ID in the HotRangesRequest. // API: PUBLIC ALPHA map<int32, NodeResponse> hot_ranges_by_node_id = 2 [ (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID", (gogoproto.customname) = "HotRangesByNodeID", (gogoproto.nullable) = false ]; } message HotRangesResponseV2 { message HotRange { int32 range_id = 1 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID", (gogoproto.customname) = "RangeID" ]; int32 node_id = 2 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; double qps = 3 [ (gogoproto.customname) = "QPS" ]; string table_name = 4; string database_name = 5; string index_name = 6; repeated int32 replica_node_ids = 7 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; } repeated HotRange ranges = 1; } message RangeRequest { int64 range_id = 1; } message RangeResponse { message NodeResponse { bool response = 1; string error_message = 2; repeated RangeInfo infos = 3 [ (gogoproto.nullable) = false ]; } // NodeID is the node that submitted all the requests. int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; int64 range_id = 2 [ (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" ]; map<int32, NodeResponse> responses_by_node_id = 3 [ (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID", (gogoproto.customname) = "ResponsesByNodeID", (gogoproto.nullable) = false ]; reserved 4; // Previously used. } // DiagnosticsRequest requests a diagnostics report. message DiagnosticsRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message StoresRequest { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. string node_id = 1; } message StoreDetails { int32 store_id = 1 [ (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" ]; // TODO(mberhault): add a lot more information about stores. eg: // - path // - settings // - encryption settings // encryption_status is a serialized // ccl/storageccl/engineccl/enginepbccl/stats.go::EncryptionStatus protobuf. bytes encryption_status = 2; // Basic file stats when encryption is enabled. // Total files/bytes. uint64 total_files = 3; uint64 total_bytes = 4; // Files/bytes using the active data key. uint64 active_key_files = 5; uint64 active_key_bytes = 6; } message StoresResponse { repeated StoreDetails stores = 1 [ (gogoproto.nullable) = false ]; } // StatementsRequest is used by both tenant and node-level // implementations to serve fan-out requests across multiple nodes or // instances. When implemented on a node, the `node_id` field refers to // the cluster nodes by their nodeID. When implemented on a tenant, the // `node_id` field refers to the instanceIDs that identify individual // tenant pods. message StatementsRequest { string node_id = 1 [(gogoproto.customname) = "NodeID"]; // If this field is set we will use the combined statements API instead. bool combined = 2; // These fields are used for the combined statements API. int64 start = 3 [(gogoproto.nullable) = true]; int64 end = 4 [(gogoproto.nullable) = true]; } message StatementsResponse { message ExtendedStatementStatisticsKey { cockroach.sql.StatementStatisticsKey key_data = 1 [(gogoproto.nullable) = false]; int32 node_id = 2 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; google.protobuf.Timestamp aggregated_ts = 3 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; // The aggregation duration. google.protobuf.Duration aggregation_interval = 4 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; } message CollectedStatementStatistics { ExtendedStatementStatisticsKey key = 1 [(gogoproto.nullable) = false]; uint64 id = 3 [(gogoproto.customname) = "ID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID"]; cockroach.sql.StatementStatistics stats = 2 [(gogoproto.nullable) = false]; } repeated CollectedStatementStatistics statements = 1 [(gogoproto.nullable) = false]; // Timestamp of the last stats reset. google.protobuf.Timestamp last_reset = 3 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; // If set and non-empty, indicates the prefix to application_name // used for statements/queries issued internally by CockroachDB. string internal_app_name_prefix = 4; message ExtendedCollectedTransactionStatistics { cockroach.sql.CollectedTransactionStatistics stats_data = 1 [(gogoproto.nullable) = false]; int32 node_id = 2 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; } // Transactions is transaction-level statistics for the collection of // statements in this response. repeated ExtendedCollectedTransactionStatistics transactions = 5 [(gogoproto.nullable) = false]; } message CombinedStatementsStatsRequest { // Unix time range for aggregated statements. int64 start = 1 [(gogoproto.nullable) = true]; int64 end = 2 [(gogoproto.nullable) = true]; } message StatementDiagnosticsReport { int64 id = 1; bool completed = 2; string statement_fingerprint = 3; int64 statement_diagnostics_id = 4 [ (gogoproto.nullable) = true ]; google.protobuf.Timestamp requested_at = 5 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; google.protobuf.Duration min_execution_latency = 6 [ (gogoproto.nullable) = false, (gogoproto.stdduration) = true ]; google.protobuf.Timestamp expires_at = 7 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; } message CreateStatementDiagnosticsReportRequest { string statement_fingerprint = 1; // MinExecutionLatency, when non-zero, indicates the minimum execution latency // of a query for which to collect the diagnostics report. In other words, if // a query executes faster than this threshold, then the diagnostics report is // not collected on it, and we will try to get a bundle the next time we see // the query fingerprint. // // NB: if MinExecutionLatency is non-zero, then all queries that match the // fingerprint will be traced until a slow enough query comes along. This // tracing might have some performance overhead. google.protobuf.Duration min_execution_latency = 2 [ (gogoproto.nullable) = false, (gogoproto.stdduration) = true ]; // ExpiresAfter, when non-zero, sets the expiration interval of this request. google.protobuf.Duration expires_after = 3 [ (gogoproto.nullable) = false, (gogoproto.stdduration) = true ]; } message CreateStatementDiagnosticsReportResponse { StatementDiagnosticsReport report = 1; } message CancelStatementDiagnosticsReportRequest { string statement_fingerprint = 1; } message CancelStatementDiagnosticsReportResponse { // canceled indicates whether the request for the given fingerprint was // actually canceled. bool canceled = 1; // error is set only if canceled is false. string error = 2; } message StatementDiagnosticsReportsRequest {} message StatementDiagnosticsReportsResponse { repeated StatementDiagnosticsReport reports = 1 [ (gogoproto.nullable) = false ]; } message StatementDiagnostics { int64 id = 1; string statement_fingerprint = 2; google.protobuf.Timestamp collected_at = 3 [ (gogoproto.nullable) = false, (gogoproto.stdtime) = true ]; reserved 4; } message StatementDiagnosticsRequest { int64 statement_diagnostics_id = 1; } message StatementDiagnosticsResponse { StatementDiagnostics diagnostics = 2; } message JobRegistryStatusRequest { string node_id = 1; } message JobRegistryStatusResponse { int32 node_id = 1 [ (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; message Job { int64 id = 1; } repeated Job running_jobs = 2; } message JobStatusRequest { int64 job_id = 1; } message JobStatusResponse { cockroach.sql.jobs.jobspb.Job job = 1; } // Request object for issuing a SQL stats reset request. message ResetSQLStatsRequest { string node_id = 1 [(gogoproto.customname) = "NodeID"]; // reset_persisted_stats specifies if the persisted SQL Stats will be reset // along with the in-memory SQL stats. bool reset_persisted_stats = 2; } // Response object returned by ResetSQLStats. message ResetSQLStatsResponse { } // Request object for issuing IndexUsageStatistics request. message IndexUsageStatisticsRequest { // node_id is the ID of the node where the stats data shall be retrieved from. // If this is left empty, the cluster-wide aggregated result will be returned. string node_id = 1 [(gogoproto.customname) = "NodeID"]; } // Response object returned by IndexUsageStatistics. message IndexUsageStatisticsResponse { repeated cockroach.sql.CollectedIndexUsageStatistics statistics = 1 [(gogoproto.nullable) = false]; // Timestamp of the last index usage stats reset. google.protobuf.Timestamp last_reset = 3 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; } // Request object for issuing TableIndexStatsRequest request. message TableIndexStatsRequest { // database is the name of the database that contains the table we're // interested in. string database = 1; // table is the name of the table that we're querying. Table may be // schema-qualified (schema.table) and each name component that contains // sql unsafe characters such as . or uppercase letters must be surrounded // in double quotes like "naughty schema".table. string table = 2; } // Response object returned by TableIndexStatsResponse. message TableIndexStatsResponse { message ExtendedCollectedIndexUsageStatistics { cockroach.sql.CollectedIndexUsageStatistics statistics = 1; // index_name is the name of the index. string index_name = 2; // index_type is the type of the index i.e. primary, secondary. string index_type = 3; // create_statement is the SQL statement that would re-create the // current index if executed. string create_statement = 4; } repeated ExtendedCollectedIndexUsageStatistics statistics = 1; // Timestamp of the last index usage stats reset. google.protobuf.Timestamp last_reset = 2 [(gogoproto.stdtime) = true]; } // Request object for issuing a index usage stats reset request. message ResetIndexUsageStatsRequest { string node_id = 1 [(gogoproto.customname) = "NodeID"]; } // Response object returned by ResetIndexUsageStatsRequest. message ResetIndexUsageStatsResponse { } service Status { // Certificates retrieves a copy of the TLS certificates. rpc Certificates(CertificatesRequest) returns (CertificatesResponse) { option (google.api.http) = { get : "/_status/certificates/{node_id}" }; } // Details retrieves details about the nodes in the cluster. rpc Details(DetailsRequest) returns (DetailsResponse) { option (google.api.http) = { get : "/_status/details/{node_id}" }; } // RegionsRequest retrieves all available regions. rpc Regions(RegionsRequest) returns (RegionsResponse) {} // Nodes returns status info for all commissioned nodes. Decommissioned nodes // are not included, except in rare cases where the node doing the // decommissioning crashed before completing the operation. In these cases, // the decommission operation can be rerun to clean up the status entry. // // API: PUBLIC ALPHA // // Don't introduce additional usages of this RPC. See #50707 for more details. // The underlying response type is something we're looking to get rid of. rpc Nodes(NodesRequest) returns (NodesResponse) { option (google.api.http) = { get : "/_status/nodes" }; } // Node retrieves details about a single node. // API: PUBLIC ALPHA rpc Node(NodeRequest) returns (server.status.statuspb.NodeStatus) { option (google.api.http) = { get : "/_status/nodes/{node_id}" }; } rpc NodesUI(NodesRequest) returns (NodesResponseExternal) { option (google.api.http) = { get : "/_status/nodes_ui" }; } rpc NodeUI(NodeRequest) returns (NodeResponse) { option (google.api.http) = { get : "/_status/nodes_ui/{node_id}" }; } // RaftDebug requests internal details about Raft. rpc RaftDebug(RaftDebugRequest) returns (RaftDebugResponse) { option (google.api.http) = { get : "/_status/raft" }; } // Ranges requests internal details about ranges on a given node. rpc Ranges(RangesRequest) returns (RangesResponse) { option (google.api.http) = { get : "/_status/ranges/{node_id}" }; } // Gossip retrieves gossip-level details about a given node. rpc Gossip(GossipRequest) returns (gossip.InfoStatus) { option (google.api.http) = { get : "/_status/gossip/{node_id}" }; } // EngineStats retrieves statistics about a storage engine. rpc EngineStats(EngineStatsRequest) returns (EngineStatsResponse) { option (google.api.http) = { get : "/_status/enginestats/{node_id}" }; } // Allocator retrieves statistics about the replica allocator. rpc Allocator(AllocatorRequest) returns (AllocatorResponse) { option (google.api.http) = { get : "/_status/allocator/node/{node_id}" }; } // AllocatorRange retrieves statistics about the replica allocator given // a specific range. rpc AllocatorRange(AllocatorRangeRequest) returns (AllocatorRangeResponse) { option (google.api.http) = { get : "/_status/allocator/range/{range_id}" }; } // ListSessions retrieves the SQL sessions across the entire cluster. rpc ListSessions(ListSessionsRequest) returns (ListSessionsResponse) { option (google.api.http) = { get : "/_status/sessions" }; } // ListLocalSessions retrieves the SQL sessions on this node. rpc ListLocalSessions(ListSessionsRequest) returns (ListSessionsResponse) { option (google.api.http) = { get : "/_status/local_sessions" }; } // CancelQuery cancels a SQL query given its ID. rpc CancelQuery(CancelQueryRequest) returns (CancelQueryResponse) { option (google.api.http) = { post : "/_status/cancel_query/{node_id}" body: "*" }; } // CancelLocalQuery cancels a SQL query running on this node given its ID. rpc CancelLocalQuery(CancelQueryRequest) returns (CancelQueryResponse) { option (google.api.http) = { post : "/_status/cancel_local_query" body: "*" }; } // ListContentionEvents retrieves the contention events across the entire // cluster. // // For SQL keys the following orderings are maintained: // - on the highest level, all IndexContentionEvents objects are ordered // according to their importance (as defined by the number of contention // events within each object). // - on the middle level, all SingleKeyContention objects are ordered by their // keys lexicographically. // - on the lowest level, all SingleTxnContention objects are ordered by the // number of times that transaction was observed to contend with other // transactions. // // For non-SQL keys the following orderings are maintained: // - on the top level, all SingleNonSQLKeyContention objects are ordered // by their keys lexicographically. // - on the bottom level, all SingleTxnContention objects are ordered by the // number of times that transaction was observed to contend with other // transactions. rpc ListContentionEvents(ListContentionEventsRequest) returns (ListContentionEventsResponse) { option (google.api.http) = { get : "/_status/contention_events" }; } // ListLocalContentionEvents retrieves the contention events on this node. // // For SQL keys the following orderings are maintained: // - on the highest level, all IndexContentionEvents objects are ordered // according to their importance (as defined by the number of contention // events within each object). // - on the middle level, all SingleKeyContention objects are ordered by their // keys lexicographically. // - on the lowest level, all SingleTxnContention objects are ordered by the // number of times that transaction was observed to contend with other // transactions. // // For non-SQL keys the following orderings are maintained: // - on the top level, all SingleNonSQLKeyContention objects are ordered // by their keys lexicographically. // - on the bottom level, all SingleTxnContention objects are ordered by the // number of times that transaction was observed to contend with other // transactions. rpc ListLocalContentionEvents(ListContentionEventsRequest) returns (ListContentionEventsResponse) { option (google.api.http) = { get : "/_status/local_contention_events" }; } // ListDistSQLFlows retrieves all of the remote flows of the DistSQL execution // that are currently running or queued on any node in the cluster. The local // flows (those that are running on the same node as the query originated on) // are not included. // // Flows are ordered lexicographically by the flow ID. rpc ListDistSQLFlows(ListDistSQLFlowsRequest) returns (ListDistSQLFlowsResponse) { option (google.api.http) = { get : "/_status/distsql_flows" }; } // ListLocalDistSQLFlows retrieves all of the remote flows of the DistSQL // execution that are currently running or queued on this node in the cluster. // Queries that originated on this node are not be included. // // Flows are ordered lexicographically by the flow ID. rpc ListLocalDistSQLFlows(ListDistSQLFlowsRequest) returns (ListDistSQLFlowsResponse) { option (google.api.http) = { get : "/_status/local_distsql_flows" }; } // CancelSessions forcefully terminates a SQL session given its ID. rpc CancelSession(CancelSessionRequest) returns (CancelSessionResponse) { option (google.api.http) = { post : "/_status/cancel_session/{node_id}" body: "*" }; } // CancelLocalSession forcefully terminates a SQL session running on this node given its ID. rpc CancelLocalSession(CancelSessionRequest) returns (CancelSessionResponse) { option (google.api.http) = { post : "/_status/cancel_local_session" body: "*" }; } // SpanStats accepts a key span and node ID, and returns a set of stats // summed from all ranges on the stores on that node which contain keys // in that span. This is designed to compute stats specific to a SQL table: // it will be called with the highest/lowest key for a SQL table, and return // information about the resources on a node used by that table. rpc SpanStats(SpanStatsRequest) returns (SpanStatsResponse) { option (google.api.http) = { post : "/_status/span" body : "*" }; } // Stacks retrieves the stack traces of all goroutines on a given node. rpc Stacks(StacksRequest) returns (JSONResponse) { option (google.api.http) = { get : "/_status/stacks/{node_id}" }; } // Profile retrieves a CPU profile on a given node. rpc Profile(ProfileRequest) returns (JSONResponse) { option (google.api.http) = { get : "/_status/profile/{node_id}" }; } // Metrics retrieves the node metrics for a given node. // // Note: this is a “reserved” API and should not be relied upon to // build external tools. No guarantee is made about its // availability and stability in external uses. rpc Metrics(MetricsRequest) returns (JSONResponse) { option (google.api.http) = { get : "/_status/metrics/{node_id}" }; } // GetFiles retrieves heap or goroutine dump files from a given node. rpc GetFiles(GetFilesRequest) returns (GetFilesResponse) { option (google.api.http) = { get : "/_status/files/{node_id}" }; } // LogFilesList retrieves a list of log files on a given node. rpc LogFilesList(LogFilesListRequest) returns (LogFilesListResponse) { option (google.api.http) = { get : "/_status/logfiles/{node_id}" }; } // LogFile retrieves a given log file. rpc LogFile(LogFileRequest) returns (LogEntriesResponse) { option (google.api.http) = { get : "/_status/logfiles/{node_id}/{file}" }; } // Logs retrieves individual log entries. rpc Logs(LogsRequest) returns (LogEntriesResponse) { option (google.api.http) = { get : "/_status/logs/{node_id}" }; } // ProblemRanges retrieves the list of “problem ranges”. rpc ProblemRanges(ProblemRangesRequest) returns (ProblemRangesResponse) { option (google.api.http) = { get : "/_status/problemranges" }; } rpc HotRanges(HotRangesRequest) returns (HotRangesResponse) { option (google.api.http) = { get : "/_status/hotranges" }; } rpc HotRanges2(HotRangesRequest) returns (HotRangesResponseV2) { option (google.api.http) = { get : "/_status/v2/hotranges" }; } rpc Range(RangeRequest) returns (RangeResponse) { option (google.api.http) = { get : "/_status/range/{range_id}" }; } rpc Diagnostics(DiagnosticsRequest) returns (cockroach.server.diagnostics.diagnosticspb.DiagnosticReport) { option (google.api.http) = { get : "/_status/diagnostics/{node_id}" }; } rpc Stores(StoresRequest) returns (StoresResponse) { option (google.api.http) = { get : "/_status/stores/{node_id}" }; } rpc Statements(StatementsRequest) returns (StatementsResponse) { option (google.api.http) = { get: "/_status/statements" }; } // Retrieve the combined in-memory and persisted statement stats by date range. rpc CombinedStatementStats(CombinedStatementsStatsRequest) returns (StatementsResponse) { option (google.api.http) = { get: "/_status/combinedstmts" }; } rpc CreateStatementDiagnosticsReport(CreateStatementDiagnosticsReportRequest) returns (CreateStatementDiagnosticsReportResponse) { option (google.api.http) = { post: "/_status/stmtdiagreports" body: "*" }; } rpc CancelStatementDiagnosticsReport(CancelStatementDiagnosticsReportRequest) returns (CancelStatementDiagnosticsReportResponse) { option (google.api.http) = { post: "/_status/stmtdiagreports/cancel" body: "*" }; } rpc StatementDiagnosticsRequests(StatementDiagnosticsReportsRequest) returns (StatementDiagnosticsReportsResponse) { option (google.api.http) = { get: "/_status/stmtdiagreports" }; } rpc StatementDiagnostics(StatementDiagnosticsRequest) returns (StatementDiagnosticsResponse) { option (google.api.http) = { get: "/_status/stmtdiag/{statement_diagnostics_id}" }; } rpc JobRegistryStatus(JobRegistryStatusRequest) returns (JobRegistryStatusResponse) { option (google.api.http) = { get : "/_status/job_registry/{node_id}" }; } rpc JobStatus(JobStatusRequest) returns (JobStatusResponse) { option (google.api.http) = { get : "/_status/job/{job_id}" }; } rpc ResetSQLStats(ResetSQLStatsRequest) returns (ResetSQLStatsResponse) { option (google.api.http) = { post: "/_status/resetsqlstats" body: "*" }; } rpc IndexUsageStatistics(IndexUsageStatisticsRequest) returns (IndexUsageStatisticsResponse) { option (google.api.http) = { get: "/_status/indexusagestatistics" }; } rpc ResetIndexUsageStats(ResetIndexUsageStatsRequest) returns (ResetIndexUsageStatsResponse) { option (google.api.http) = { post: "/_status/resetindexusagestats" body: "*" }; } // TableIndexStats retrieves index stats for a table. rpc TableIndexStats(TableIndexStatsRequest) returns (TableIndexStatsResponse) { option (google.api.http) = { get: "/_status/databases/{database}/tables/{table}/indexstats" }; } }