Permalink
Cannot retrieve contributors at this time
Join GitHub today
GitHub is home to over 28 million developers working together to host and review code, manage projects, and build software together.
Sign up
Fetching contributors…
| // Copyright 2016 The Cockroach Authors. | |
| // | |
| // Licensed 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. | |
| // | |
| // Processor definitions for distributed SQL APIs. See | |
| // docs/RFCS/distributed_sql.md. | |
| // All the concepts here are "physical plan" concepts. | |
| syntax = "proto2"; | |
| package cockroach.sql.distsqlrun; | |
| option go_package = "distsqlrun"; | |
| import "roachpb/api.proto"; | |
| import "roachpb/csv.proto"; | |
| import "roachpb/data.proto"; | |
| import "roachpb/errors.proto"; | |
| import "sql/sqlbase/structured.proto"; | |
| import "sql/sqlbase/encoded_datum.proto"; | |
| import "sql/sqlbase/join_type.proto"; | |
| import "sql/distsqlrun/data.proto"; | |
| import "util/hlc/timestamp.proto"; | |
| import "gogoproto/gogo.proto"; | |
| // Each processor has the following components: | |
| // - one or more input synchronizers; each one merges rows between one or more | |
| // input streams; | |
| // | |
| // - a processor "core" which encapsulates the inner logic of each processor; | |
| // | |
| // - a post-processing stage which allows "inline" post-processing on results | |
| // (like projection or filtering); | |
| // | |
| // - one or more output synchronizers; each one directs rows to one or more | |
| // output streams. | |
| // | |
| // | |
| // == Internal columns == | |
| // | |
| // The core outputs rows of a certain schema to the post-processing stage. We | |
| // call this the "internal schema" (or "internal columns") and it differs for | |
| // each type of core. Column indices in a PostProcessSpec refers to these | |
| // internal columns. Some columns may be unused by the post-processing stage; | |
| // processor implementations are internally optimized to not produce values for | |
| // such unneded columns. | |
| message ProcessorSpec { | |
| // In most cases, there is one input. | |
| repeated InputSyncSpec input = 1 [(gogoproto.nullable) = false]; | |
| optional ProcessorCoreUnion core = 2 [(gogoproto.nullable) = false]; | |
| optional PostProcessSpec post = 4 [(gogoproto.nullable) = false]; | |
| // In most cases, there is one output. | |
| repeated OutputRouterSpec output = 3 [(gogoproto.nullable) = false]; | |
| // An optional identifier that can be used to correlate processors that are | |
| // part of the same stage (e.g. multiple joiners that are part of a | |
| // distributed join). This has no consequence on the running of flows, but is | |
| // useful for plan diagrams. | |
| optional int32 stage_id = 5 [(gogoproto.nullable) = false, | |
| (gogoproto.customname) = "StageID"]; | |
| } | |
| // PostProcessSpec describes the processing required to obtain the output | |
| // (filtering, projection). It operates on the internal schema of the processor | |
| // (see ProcessorSpec). | |
| message PostProcessSpec { | |
| // A filtering expression which references the internal columns of the | |
| // processor via ordinal references (@1, @2, etc). | |
| optional Expression filter = 1 [(gogoproto.nullable) = false]; | |
| // If true, output_columns describes a projection. Used to differentiate | |
| // between an empty projection and no projection. | |
| // | |
| // Cannot be set at the same time with render expressions. | |
| optional bool projection = 2 [(gogoproto.nullable) = false]; | |
| // The output columns describe a projection on the internal set of columns; | |
| // only the columns in this list will be emitted. | |
| // | |
| // Can only be set if projection is true. Cannot be set at the same time with | |
| // render expressions. | |
| repeated uint32 output_columns = 3 [packed = true]; | |
| // If set, the output is the result of rendering these expressions. The | |
| // expressions reference the internal columns of the processor. | |
| // | |
| // Cannot be set at the same time with output columns. | |
| repeated Expression render_exprs = 4 [(gogoproto.nullable) = false]; | |
| // If nonzero, the first <offset> rows will be suppressed. | |
| optional uint64 offset = 5 [(gogoproto.nullable) = false]; | |
| // If nonzero, the processor will stop after emitting this many rows. The rows | |
| // suppressed by <offset>, if any, do not count towards this limit. | |
| optional uint64 limit = 6 [(gogoproto.nullable) = false]; | |
| } | |
| message ProcessorCoreUnion { | |
| option (gogoproto.onlyone) = true; | |
| optional NoopCoreSpec noop = 1; | |
| optional TableReaderSpec tableReader = 2; | |
| optional JoinReaderSpec joinReader = 3; | |
| optional SorterSpec sorter = 4; | |
| optional AggregatorSpec aggregator = 5; | |
| optional DistinctSpec distinct = 7; | |
| optional MergeJoinerSpec mergeJoiner = 8; | |
| optional HashJoinerSpec hashJoiner = 9; | |
| optional ValuesCoreSpec values = 10; | |
| optional BackfillerSpec backfiller = 11; | |
| optional ReadCSVSpec readCSV = 13; | |
| optional SSTWriterSpec SSTWriter = 14; | |
| optional CSVWriterSpec CSVWriter = 20; | |
| optional SamplerSpec Sampler = 15; | |
| optional SampleAggregatorSpec SampleAggregator = 16; | |
| optional InterleavedReaderJoinerSpec interleavedReaderJoiner = 17; | |
| optional MetadataTestSenderSpec metadataTestSender = 18; | |
| optional MetadataTestReceiverSpec metadataTestReceiver = 19; | |
| reserved 6, 12; | |
| } | |
| // NoopCoreSpec indicates a "no-op" processor core. This is used when we just | |
| // need post-processing or when only a synchronizer is required (e.g. at the | |
| // final endpoint). | |
| message NoopCoreSpec { | |
| } | |
| message MetadataTestSenderSpec { | |
| optional string id = 1 [(gogoproto.nullable) = false, | |
| (gogoproto.customname) = "ID"]; | |
| } | |
| message MetadataTestReceiverSpec { | |
| repeated string sender_ids = 1 [(gogoproto.customname) = "SenderIDs"]; | |
| } | |
| // ValuesCoreSpec is the core of a processor that has no inputs and generates | |
| // "pre-canned" rows. This is not intended to be used for very large datasets. | |
| message ValuesCoreSpec { | |
| // There is one DatumInfo for each element in a row. | |
| repeated DatumInfo columns = 1 [(gogoproto.nullable) = false]; | |
| // Each raw block encodes one or more data rows; each datum is encoded | |
| // according to the corresponding DatumInfo. | |
| repeated bytes raw_bytes = 2; | |
| } | |
| message TableReaderSpan { | |
| // TODO(radu): the dist_sql APIs should be agnostic to how we map tables to | |
| // KVs. The span should be described as starting and ending lists of values | |
| // for a prefix of the index columns, along with inclusive/exclusive flags. | |
| optional roachpb.Span span = 1 [(gogoproto.nullable) = false]; | |
| } | |
| // TableReaderSpec is the specification for a "table reader". A table reader | |
| // performs KV operations to retrieve rows for a table and outputs the desired | |
| // columns of the rows that pass a filter expression. | |
| // | |
| // The "internal columns" of a TableReader (see ProcessorSpec) are all the | |
| // columns of the table. Internally, only the values for the columns needed by | |
| // the post-processing stage are be populated. If is_check is set, the | |
| // TableReader will run additional data checking procedures and the | |
| // "internal columns" are: | |
| // - Error type (string). | |
| // - Primary key as a string, if it was obtainable. | |
| // - JSON of all decoded column values. | |
| message TableReaderSpec { | |
| optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; | |
| // If 0, we use the primary index. If non-zero, we use the index_idx-th index, | |
| // i.e. table.indexes[index_idx-1] | |
| optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; | |
| optional bool reverse = 3 [(gogoproto.nullable) = false]; | |
| repeated TableReaderSpan spans = 4 [(gogoproto.nullable) = false]; | |
| // A hint for how many rows the consumer of the table reader output might | |
| // need. This is used to size the initial KV batches to try to avoid reading | |
| // many more rows than needed by the processor receiving the output. | |
| // | |
| // Not used if there is a limit set in the PostProcessSpec of this processor | |
| // (that value will be used for sizing batches instead). | |
| optional int64 limit_hint = 5 [(gogoproto.nullable) = false]; | |
| // Indicates whether the TableReader is being run as an exhaustive | |
| // check. This is only true during SCRUB commands. | |
| optional bool is_check = 6 [(gogoproto.nullable) = false]; | |
| } | |
| // JoinReaderSpec is the specification for a "join reader". A join reader | |
| // performs KV operations to retrieve specific rows that correspond to the | |
| // values in the input stream (join by lookup). | |
| // | |
| // The "internal columns" of a JoinReader (see ProcessorSpec) are either the | |
| // columns of the table or the concatenation of the columns of the input stream | |
| // with the table columns, depending on the lookup columns specified. | |
| // Internally, only the values for the columns needed by the post-processing | |
| // stage are be populated. | |
| // | |
| // Example: | |
| // Input stream columns: | a | b | Table columns: | c | d | e | | |
| // | |
| // If performing a lookup join on a = c (lookup columns is [0]): | |
| // Internal columns: | a | b | c | d | e | | |
| // | |
| // If performing an index join (where a = c and b = d) (lookup columns is []): | |
| // Internal columns: | c | d | e | | |
| message JoinReaderSpec { | |
| optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; | |
| // If 0, we use the primary index; each row in the input stream has a value | |
| // for each primary key. | |
| // TODO(radu): figure out the correct semantics when joining with an index. | |
| optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; | |
| // Column indexes in the input stream specifying the columns which match with | |
| // the index columns. These are the equality columns of the join. | |
| // If empty (index join), the start of the input stream schema is assumed to | |
| // match the index columns. The joinReader will perform an index join and the | |
| // "internal columns" will be the columns of the table. | |
| // If populated (lookup join), the `joinReader` will perform a lookup join | |
| // and the "internal columns" will be the concatenation of the input stream | |
| // columns followed by the table columns. | |
| repeated uint32 lookup_columns = 3 [packed = true]; | |
| // "ON" expression (in addition to the equality constraints captured by the | |
| // orderings). Assuming that the left stream has N columns and the right | |
| // stream has M columns, in this expression variables @1 to @N refer to | |
| // columns of the left stream and variables @N to @(N+M) refer to columns in | |
| // the right stream. | |
| // Having "ON" expression implies no merged columns. | |
| optional Expression on_expr = 4 [(gogoproto.nullable) = false]; | |
| } | |
| // SorterSpec is the specification for a "sorting aggregator". A sorting | |
| // processor sorts elements in the input stream providing a certain output | |
| // order guarantee regardless of the input ordering. The output ordering is | |
| // according to a configurable set of columns. | |
| // | |
| // The "internal columns" of a Sorter (see ProcessorSpec) are the same as the | |
| // input columns. | |
| message SorterSpec { | |
| optional Ordering output_ordering = 1 [(gogoproto.nullable) = false]; | |
| // Ordering match length, specifying that the input is already sorted by the | |
| // first 'n' output ordering columns, can be optionally specified for | |
| // possible speed-ups taking advantage of the partial orderings. | |
| optional uint32 ordering_match_len = 2 [(gogoproto.nullable) = false]; | |
| } | |
| message DistinctSpec { | |
| // The ordered columns in the input stream can be optionally specified for | |
| // possible optimizations. The specific ordering (ascending/descending) of | |
| // the column itself is not important nor is the order in which the columns | |
| // are specified. | |
| repeated uint32 ordered_columns = 1; | |
| // The distinct columns in the input stream are those columns on which we | |
| // check for distinct rows. If A,B,C are in distinct_columns and there is a | |
| // 4th column D which is not included in distinct_columns, its values are not | |
| // considered, so rows A1,B1,C1,D1 and A1,B1,C1,D2 are considered equal and | |
| // only one of them (the first) is output. | |
| repeated uint32 distinct_columns = 2; | |
| } | |
| // MergeJoinerSpec is the specification for a merge join processor. The processor | |
| // has two inputs and one output. The inputs must have the same ordering on the | |
| // columns that have equality constraints. For example: | |
| // SELECT * FROM T1 INNER JOIN T2 ON T1.C1 = T2.C5 AND T1.C2 = T2.C4 | |
| // | |
| // To perform a merge join, the streams corresponding to T1 and T2 must have the | |
| // same ordering on columns C1, C2 and C5, C4 respectively. For example: C1+,C2- | |
| // and C5+,C4-. | |
| // | |
| // The "internal columns" of a MergeJoiner (see ProcessorSpec) are the | |
| // concatenation of left input columns and right input columns. If the left | |
| // input has N columns and the right input has M columns, the first N columns | |
| // contain values from the left side and the following M columns contain values | |
| // from the right side. | |
| message MergeJoinerSpec { | |
| // The streams must be ordered according to the columns that have equality | |
| // constraints. The first column of the left ordering is constrained to be | |
| // equal to the first column in the right ordering and so on. The ordering | |
| // lengths and directions must match. | |
| // In the example above, left ordering describes C1+,C2- and right ordering | |
| // describes C5+,C4-. | |
| optional Ordering left_ordering = 1 [(gogoproto.nullable) = false]; | |
| optional Ordering right_ordering = 2 [(gogoproto.nullable) = false]; | |
| // "ON" expression (in addition to the equality constraints captured by the | |
| // orderings). Assuming that the left stream has N columns and the right | |
| // stream has M columns, in this expression ordinal references @1 to @N refer | |
| // to columns of the left stream and variables @(N+1) to @(N+M) refer to | |
| // columns in the right stream. | |
| optional Expression on_expr = 5 [(gogoproto.nullable) = false]; | |
| optional sqlbase.JoinType type = 6 [(gogoproto.nullable) = false]; | |
| // NullEquality indicates that NULL = NULL should be considered true. | |
| // This allows OUTER JOINs to consider NULL values meaningfully. An | |
| // example of this is during SCRUB checks on secondary indexes. | |
| optional bool null_equality = 7 [(gogoproto.nullable) = false]; | |
| } | |
| // HashJoinerSpec is the specification for a hash join processor. The processor | |
| // has two inputs and one output. | |
| // | |
| // The processor works by reading the entire right input and putting it in a hash | |
| // table. Thus, there is no guarantee on the ordering of results that stem only | |
| // from the right input (in the case of RIGHT_OUTER, FULL_OUTER). However, it is | |
| // guaranteed that results that involve the left stream preserve the ordering; | |
| // i.e. all results that stem from left row (i) precede results that stem from | |
| // left row (i+1). | |
| // | |
| // The "internal columns" of a HashJoiner (see ProcessorSpec) are the | |
| // concatenation of merged columns (if present), left input columns and right | |
| // input columns. Each merged column corresponds to a left and a right equality | |
| // column; its value is the value on the left if it is not NULL, otherwise it is | |
| // the value on the right. There are either zero or | |
| // E=len(left_eq_columns)=len(right_eq_columns) merged columns. | |
| // | |
| // If the left input has N columns and the right input has M columns, the | |
| // first N columns contain values from the left side and the following M columns | |
| // contain values from the right side. If merged columns are present, they | |
| // occupy first E positions followed by N values from the left side and M values | |
| // from the right side. | |
| message HashJoinerSpec { | |
| // The join constraints certain columns from the left stream to equal | |
| // corresponding columns on the right stream. These must have the same length. | |
| repeated uint32 left_eq_columns = 1 [packed = true]; | |
| repeated uint32 right_eq_columns = 2 [packed = true]; | |
| // "ON" expression (in addition to the equality constraints captured by the | |
| // orderings). Assuming that the left stream has N columns and the right | |
| // stream has M columns, in this expression variables @1 to @N refer to | |
| // columns of the left stream and variables @N to @(N+M) refer to columns in | |
| // the right stream. | |
| // Having "ON" expression implies no merged columns. | |
| optional Expression on_expr = 5 [(gogoproto.nullable) = false]; | |
| optional sqlbase.JoinType type = 6 [(gogoproto.nullable) = false]; | |
| // DEPRECATED | |
| // | |
| // Extra merged columns that are added in case of OUTER JOINS. These | |
| // columns occupy first positions in a row amd it's the left value if it's not | |
| // NULL, otherwise it's the right value. In INNER JOIN case no merged columns are | |
| // needed since left stream values are guaranteed to be not NULL. | |
| // | |
| // This has been deprecated; the distsqlrun layer still supports it for | |
| // backward compatibility during upgrade. | |
| optional bool merged_columns = 7 [(gogoproto.nullable) = false]; | |
| } | |
| // AggregatorSpec is the specification for an "aggregator" (processor core | |
| // type, not the logical plan computation stage). An aggregator performs | |
| // 'aggregation' in the SQL sense in that it groups rows and computes an aggregate | |
| // for each group. The group is configured using the group key. The aggregator | |
| // can be configured with one or more aggregation functions. | |
| // | |
| // The "internal columns" of an Aggregator map 1-1 to the aggregations. | |
| message AggregatorSpec { | |
| // These mirror the aggregate functions supported by sql/parser. See | |
| // sql/parser/aggregate_builtins.go. | |
| enum Func { | |
| // The identity function is set to be the default zero-value function, | |
| // returning the last value added. | |
| IDENT = 0; | |
| AVG = 1; | |
| BOOL_AND = 2; | |
| BOOL_OR = 3; | |
| CONCAT_AGG = 4; | |
| COUNT = 5; | |
| MAX = 7; | |
| MIN = 8; | |
| STDDEV = 9; | |
| SUM = 10; | |
| SUM_INT = 11; | |
| VARIANCE = 12; | |
| XOR_AGG = 13; | |
| COUNT_ROWS = 14; | |
| SQRDIFF = 15; | |
| FINAL_VARIANCE = 16; | |
| FINAL_STDDEV = 17; | |
| ARRAY_AGG = 18; | |
| JSON_AGG = 19; | |
| // JSONB_AGG is an alias for JSON_AGG, they do the same thing. | |
| JSONB_AGG = 20; | |
| } | |
| message Aggregation { | |
| optional Func func = 1 [(gogoproto.nullable) = false]; | |
| // Aggregation functions with distinct = true functions like you would | |
| // expect '<FUNC> DISTINCT' to operate, the default behavior would be | |
| // the '<FUNC> ALL' operation. | |
| optional bool distinct = 2 [(gogoproto.nullable) = false]; | |
| // The column index specifies the argument(s) to the aggregator function. | |
| // | |
| // Most aggregations take one argument | |
| // COUNT_ROWS takes no arguments. | |
| // FINAL_STDDEV and FINAL_VARIANCE take three arguments (SQRDIFF, SUM, | |
| // COUNT). | |
| repeated uint32 col_idx = 5; | |
| // If set, this column index specifies a boolean argument; rows for which | |
| // this value is not true don't contribute to this aggregation. This enables | |
| // the filter clause, e.g.: | |
| // SELECT SUM(x) FILTER (WHERE y > 1), SUM(x) FILTER (WHERE y < 1) FROM t | |
| optional uint32 filter_col_idx = 4; | |
| reserved 3; | |
| } | |
| // The group key is a subset of the columns in the input stream schema on the | |
| // basis of which we define our groups. | |
| repeated uint32 group_cols = 2 [packed = true]; | |
| repeated Aggregation aggregations = 3 [(gogoproto.nullable) = false]; | |
| // A subset of the GROUP BY columns which are ordered in the input. | |
| repeated uint32 ordered_group_cols = 4 [packed = true]; | |
| } | |
| // BackfillerSpec is the specification for a "schema change backfiller". | |
| // The created backfill processor runs a backfill for the first mutations in | |
| // the table descriptor mutation list with the same mutation id and type. | |
| // A backfiller processor performs KV operations to retrieve rows for a | |
| // table and backfills the new indexes/columns contained in the table | |
| // descriptor. It checkpoints its progress by updating the table | |
| // descriptor in the database, and doesn't emit any rows nor support | |
| // any post-processing. | |
| message BackfillerSpec { | |
| enum Type { | |
| Invalid = 0; | |
| Column = 1; | |
| Index = 2; | |
| } | |
| optional Type type = 1 [(gogoproto.nullable) = false]; | |
| optional sqlbase.TableDescriptor table = 2 [(gogoproto.nullable) = false]; | |
| // Sections of the table to be backfilled. | |
| repeated TableReaderSpan spans = 3 [(gogoproto.nullable) = false]; | |
| // Run the backfill for approximately this duration. | |
| // The backfill will always process at least one backfill chunk. | |
| optional int64 duration = 4 [(gogoproto.nullable) = false, (gogoproto.casttype) = "time.Duration"]; | |
| // The backfill involves a complete table scan in chunks, | |
| // where each chunk is a transactional read of a set of rows | |
| // along with a backfill for the rows. This is the maximum number | |
| // of entries backfilled per chunk. | |
| optional int64 chunk_size = 5 [(gogoproto.nullable) = false]; | |
| // Any other (leased) table descriptors necessary for the | |
| // backfiller to do its job, such as the descriptors for tables with fk | |
| // relationships to the table being modified. | |
| repeated sqlbase.TableDescriptor other_tables = 6 [(gogoproto.nullable) = false]; | |
| // The timestamp to perform index backfill historical scans at. | |
| optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false]; | |
| } | |
| // FlowSpec describes a "flow" which is a subgraph of a distributed SQL | |
| // computation consisting of processors and streams. | |
| message FlowSpec { | |
| optional bytes flow_id = 1 [(gogoproto.nullable) = false, | |
| (gogoproto.customname) = "FlowID", | |
| (gogoproto.customtype) = "FlowID"]; | |
| // The NodeID of the gateway that planned this Flow. Used for debugging. | |
| optional int32 gateway = 3 [(gogoproto.nullable) = false, | |
| (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; | |
| repeated ProcessorSpec processors = 2 [(gogoproto.nullable) = false]; | |
| } | |
| // JobProgress identifies the job to report progress on. This reporting | |
| // happens outside this package. | |
| message JobProgress { | |
| optional int64 job_id = 1 [(gogoproto.nullable) = false, | |
| (gogoproto.customname) = "JobID"]; | |
| // contribution is the percent of work of the total this processor will | |
| // process. | |
| optional float contribution = 2 [(gogoproto.nullable) = false]; | |
| // slot is the index into the job details for this processor's completion. | |
| optional int32 slot = 3 [(gogoproto.nullable) = false]; | |
| } | |
| // ReadCSVSpec is the specification for a processor that reads a CSV | |
| // file at uri with descriptor table_desc, specified delimiter (comma), | |
| // optional comment rune, and optional nullif string (which is nullable to | |
| // differentiate between not set (nil) and the empty string (which could be | |
| // used as the null marker). It outputs rows that are a sampling of the file | |
| // at a rate of (row size) / sample_size. | |
| // See ccs/sqlccl/csv.go for implementation. | |
| message ReadCSVSpec { | |
| optional roachpb.CSVOptions options = 1 [(gogoproto.nullable) = false]; | |
| // sample_size is the rate at which to output rows, based on an input row's size. | |
| optional int32 sample_size = 2 [(gogoproto.nullable) = false]; | |
| // table_desc is the table descriptor of the CSV file. | |
| optional sqlbase.TableDescriptor table_desc = 3 [(gogoproto.nullable) = false]; | |
| // uri is a storageccl.ExportStorage URI pointing to the CSV files to be | |
| // read. The map key must be unique across the entire IMPORT job. | |
| map<int32, string> uri = 7; | |
| optional JobProgress progress = 6 [(gogoproto.nullable) = false]; | |
| reserved 4; | |
| reserved 5; | |
| } | |
| // SSTWriterSpec is the specification for a processor that consumes rows, uses | |
| // tempStorage to sort them, then writes them to SST files at uri. walltime is | |
| // used as the MVCC timestamp. It outputs one row per span containing the file | |
| // name, size, checksum, observed start and end keys. See ccs/sqlccl/csv.go | |
| // for implementation. | |
| message SSTWriterSpec { | |
| message SpanName { | |
| // name is the file name that will be written by the export store. | |
| optional string name = 1 [(gogoproto.nullable) = false]; | |
| // end is the end key of a span. | |
| optional bytes end = 2; | |
| } | |
| // destination as a storageccl.ExportStorage URI pointing to an export store | |
| // location (directory). | |
| optional string destination = 1 [(gogoproto.nullable) = false]; | |
| // walltimeNanos is the MVCC time at which the created KVs will be written. | |
| optional int64 walltimeNanos = 3 [(gogoproto.nullable) = false]; | |
| // spans is an array of span boundaries and corresponding filenames. | |
| repeated SpanName spans = 4 [(gogoproto.nullable) = false]; | |
| optional JobProgress progress = 5 [(gogoproto.nullable) = false]; | |
| reserved 2; | |
| } | |
| // CSVWriterSpec is the specification for a processor that consumes rows and | |
| // writes them to CSV files at uri. It outputs one row per span containing the | |
| // file name, rowcount and size, checksum. | |
| message CSVWriterSpec { | |
| // destination as a storageccl.ExportStorage URI pointing to an export store | |
| // location (directory). | |
| optional string destination = 1 [(gogoproto.nullable) = false]; | |
| optional string name_pattern = 2 [(gogoproto.nullable) = false]; | |
| optional int32 delimiter = 3 [(gogoproto.nullable) = false]; | |
| // chunk_rows = 0 indicated no limit. | |
| optional int64 chunk_rows = 4 [(gogoproto.nullable) = false]; | |
| } | |
| enum SketchType { | |
| // This is the github.com/axiomhq/hyperloglog binary format | |
| // (as of commit 730eea1) for a sketch with precision 14. | |
| HLL_PLUS_PLUS_V1 = 0; | |
| } | |
| // SketchSpec contains the specification for a generated statistic. | |
| message SketchSpec { | |
| optional SketchType sketch_type = 1 [(gogoproto.nullable) = false]; | |
| // Each value is an index identifying a column in the input stream. | |
| // TODO(radu): currently only one column is supported. | |
| repeated uint32 columns = 2; | |
| // If set, we generate a histogram for the first column in the sketch. | |
| // Only used by the SampleAggregator. | |
| optional bool generate_histogram = 3 [(gogoproto.nullable) = false]; | |
| // Controls the maximum number of buckets in the histogram. | |
| // Only used by the SampleAggregator. | |
| optional uint32 histogram_max_buckets = 4 [(gogoproto.nullable) = false]; | |
| // Only used by the SampleAggregator. | |
| optional string stat_name = 5 [(gogoproto.nullable) = false]; | |
| } | |
| // SamplerSpec is the specification of a "sampler" processor which | |
| // returns a sample (random subset) of the input columns and computes | |
| // cardinality estimation sketches on sets of columns. | |
| // | |
| // The sampler is configured with a sample size and sets of columns | |
| // for the sketches. It produces one row with global statistics, one | |
| // row with sketch information for each sketch plus at most | |
| // sample_size sampled rows. | |
| // | |
| // The following method is used to do reservoir sampling: we generate a | |
| // "rank" for each row, which is just a random, uniformly distributed | |
| // 64-bit value. The rows with the smallest <sample_size> ranks are selected. | |
| // This method is chosen because it allows to combine sample sets very easily. | |
| // | |
| // The internal schema of the processor is formed of two column | |
| // groups: | |
| // 1. sampled row columns: | |
| // - columns that map 1-1 to the columns in the input (same | |
| // schema as the input). | |
| // - an INT column with the "rank" of the row; this is a random value | |
| // associated with the row (necessary for combining sample sets). | |
| // 2. sketch columns: | |
| // - an INT column indicating the sketch index | |
| // (0 to len(sketches) - 1). | |
| // - an INT column indicating the number of rows processed | |
| // - an INT column indicating the number of NULL values | |
| // on the first column of the sketch. | |
| // - a BYTES column with the binary sketch data (format | |
| // dependent on the sketch type). | |
| // Rows have NULLs on either all the sampled row columns or on all the | |
| // sketch columns. | |
| message SamplerSpec { | |
| repeated SketchSpec sketches = 1 [(gogoproto.nullable) = false]; | |
| optional uint32 sample_size = 2 [(gogoproto.nullable) = false]; | |
| } | |
| // SampleAggregatorSpec is the specification of a processor that aggregates the | |
| // results from multiple sampler processors and writes out the statistics to | |
| // system.table_statistics. | |
| // | |
| // The input schema it expects matches the output schema of a sampler spec (see | |
| // the comment for SamplerSpec for all the details): | |
| // 1. sampled row columns: | |
| // - sampled columns | |
| // - row rank | |
| // 2. sketch columns: | |
| // - sketch index | |
| // - number of rows processed | |
| // - number of NULL values encountered on the first column of the sketch | |
| // - binary sketch data | |
| message SampleAggregatorSpec { | |
| repeated SketchSpec sketches = 1 [(gogoproto.nullable) = false]; | |
| // The processor merges reservoir sample sets into a single | |
| // sample set of this size. This must match the sample size | |
| // used for each Sampler. | |
| optional uint32 sample_size = 2 [(gogoproto.nullable) = false]; | |
| // The i-th value indicates the ColumnID of the i-th sampled row column. | |
| // These are necessary for writing out the statistic data. | |
| repeated uint32 sampled_column_ids = 3 [ | |
| (gogoproto.customname) = "SampledColumnIDs", | |
| (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ColumnID" | |
| ]; | |
| optional uint32 table_id = 4 [ | |
| (gogoproto.nullable) = false, | |
| (gogoproto.customname) = "TableID", | |
| (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" | |
| ]; | |
| } | |
| // InterleavedReaderJoinerSpec is the specification for a processor that performs | |
| // KV operations to retrieve rows from 2+ tables from an interleaved hierarchy, | |
| // performs intermediate filtering on rows from each table, and performs a | |
| // join on the rows from the 2+ tables. | |
| // | |
| // Limitations: the InterleavedReaderJoiner currently supports only equality INNER joins | |
| // on the full interleave prefix. | |
| // See https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20171025_interleaved_table_joins.md. | |
| // | |
| // The "internal columns" of an InterleavedReaderJoiner are the | |
| // concatenation of left input columns and right input columns. If the left | |
| // table has N columns and the right table has M columns, the first N columns | |
| // contain values from the left table and the following M columns contain values | |
| // from the right table. | |
| message InterleavedReaderJoinerSpec { | |
| message Table { | |
| optional sqlbase.TableDescriptor desc = 1 [(gogoproto.nullable) = false]; | |
| // If 0, we use the primary index. If non-zero, we use the index_idx-th index, | |
| // i.e. desc.indexes[index_idx-1] | |
| optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; | |
| // The PostProcessSpecs of the corresponding TableReaderSpecs of each table | |
| // are fed as arguments to InterleavedReaderJoiner. | |
| // | |
| // This is required to properly post-process the rows (i.e. filtering and | |
| // projections) after reading from the table but before joining. | |
| // It may be necessary to modify/introduce additional intermediate filters | |
| // for correctness (see comment above 'spans' under | |
| // InterleavedReaderJoinerSpec). | |
| optional PostProcessSpec post = 3 [(gogoproto.nullable) = false]; | |
| // The tables must be ordered according to the columns that have equality | |
| // constraints. The first column of the first table's ordering is constrained | |
| // to be equal to the first column in the second table's ordering and so on | |
| // for the other tables and their corresponding columns. | |
| optional Ordering ordering = 4 [(gogoproto.nullable) = false]; | |
| // The span covering the rows from this table to join. Note the | |
| // InterleavedReaderJoiner processor will taking the union of all spans across | |
| // all tables to do a single pass-through scan. InterleavedReaderJoiner will | |
| // then check if a given row for a table is within any of its spans. | |
| // There must exist at least one non-empty set of spans for some table. | |
| repeated TableReaderSpan spans = 5 [(gogoproto.nullable) = false]; | |
| } | |
| // For the common case of two tables, table at index 0 is the left/parent | |
| // table and table at index 1 is the right/child table. | |
| repeated Table tables = 1 [(gogoproto.nullable) = false]; | |
| // Reader component | |
| optional bool reverse = 2 [(gogoproto.nullable) = false]; | |
| // A hint for how many joined rows from the tables the consumer of the | |
| // interleavedReaderJoiner might need. This is used to size the initial KV | |
| // batches to try to avoid reading many more rows than needed by the | |
| // processor receiving the output. | |
| // // Not used if there is a limit set in the PostProcessSpec of this processor // (that value will be used for sizing batches instead). | |
| optional int64 limit_hint = 3 [(gogoproto.nullable) = false]; | |
| // Joiner component | |
| // "ON" expression (in addition to the equality constraints captured by the | |
| // orderings). Assuming that the left table has N columns and the second | |
| // table stream has M columns, in this expression ordinal references @1 to @N | |
| // refer to columns of the left table and variables @(N+1) to @(N+M) refer to | |
| // columns in the right table. | |
| optional Expression on_expr = 4 [(gogoproto.nullable) = false]; | |
| optional sqlbase.JoinType type = 5 [(gogoproto.nullable) = false]; | |
| } |