github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/execinfrapb/processors.proto (about) 1 // Copyright 2016 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 // 11 // Processor definitions for distributed SQL APIs. See 12 // docs/RFCS/distributed_sql.md. 13 // All the concepts here are "physical plan" concepts. 14 15 syntax = "proto2"; 16 // Beware! This package name must not be changed, even though it doesn't match 17 // the Go package name, because it defines the Protobuf message names which 18 // can't be changed without breaking backward compatibility. 19 package cockroach.sql.distsqlrun; 20 option go_package = "execinfrapb"; 21 22 import "sql/execinfrapb/data.proto"; 23 import "sql/execinfrapb/processors_base.proto"; 24 import "sql/execinfrapb/processors_sql.proto"; 25 import "sql/execinfrapb/processors_bulk_io.proto"; 26 import "sql/execinfrapb/processors_changefeeds.proto"; 27 import "sql/execinfrapb/processors_table_stats.proto"; 28 import "gogoproto/gogo.proto"; 29 30 // Each processor has the following components: 31 // - one or more input synchronizers; each one merges rows between one or more 32 // input streams; 33 // 34 // - a processor "core" which encapsulates the inner logic of each processor; 35 // 36 // - a post-processing stage which allows "inline" post-processing on results 37 // (like projection or filtering); 38 // 39 // - one or more output synchronizers; each one directs rows to one or more 40 // output streams. 41 // 42 // 43 // == Internal columns == 44 // 45 // The core outputs rows of a certain schema to the post-processing stage. We 46 // call this the "internal schema" (or "internal columns") and it differs for 47 // each type of core. Column indices in a PostProcessSpec refers to these 48 // internal columns. Some columns may be unused by the post-processing stage; 49 // processor implementations are internally optimized to not produce values for 50 // such unneeded columns. 51 message ProcessorSpec { 52 // In most cases, there is one input. 53 repeated InputSyncSpec input = 1 [(gogoproto.nullable) = false]; 54 55 optional ProcessorCoreUnion core = 2 [(gogoproto.nullable) = false]; 56 57 optional PostProcessSpec post = 4 [(gogoproto.nullable) = false]; 58 59 // In most cases, there is one output. 60 repeated OutputRouterSpec output = 3 [(gogoproto.nullable) = false]; 61 62 // An optional identifier that can be used to correlate processors that are 63 // part of the same stage (e.g. multiple joiners that are part of a 64 // distributed join). This has no consequence on the running of flows, but is 65 // useful for plan diagrams. 66 optional int32 stage_id = 5 [(gogoproto.nullable) = false, 67 (gogoproto.customname) = "StageID"]; 68 69 // processor_id uniquely identifies a processor within a physical plan. This 70 // has no consequence on the running of flows, but is useful for plan 71 // diagrams and linking information like tracing spans and log messages to 72 // processors. 73 optional int32 processor_id = 6 [(gogoproto.nullable) = false, 74 (gogoproto.customname) = "ProcessorID"]; 75 } 76 77 message ProcessorCoreUnion { 78 option (gogoproto.onlyone) = true; 79 80 optional NoopCoreSpec noop = 1; 81 optional TableReaderSpec tableReader = 2; 82 optional JoinReaderSpec joinReader = 3; 83 optional SorterSpec sorter = 4; 84 optional AggregatorSpec aggregator = 5; 85 optional DistinctSpec distinct = 7; 86 optional MergeJoinerSpec mergeJoiner = 8; 87 optional HashJoinerSpec hashJoiner = 9; 88 optional ValuesCoreSpec values = 10; 89 optional BackfillerSpec backfiller = 11; 90 optional ReadImportDataSpec readImport = 13; 91 reserved 14; 92 optional CSVWriterSpec CSVWriter = 20; 93 optional SamplerSpec Sampler = 15; 94 optional SampleAggregatorSpec SampleAggregator = 16; 95 optional InterleavedReaderJoinerSpec interleavedReaderJoiner = 17; 96 optional MetadataTestSenderSpec metadataTestSender = 18; 97 optional MetadataTestReceiverSpec metadataTestReceiver = 19; 98 optional ZigzagJoinerSpec zigzagJoiner = 21; 99 optional ProjectSetSpec projectSet = 22; 100 optional WindowerSpec windower = 23; 101 optional LocalPlanNodeSpec localPlanNode = 24; 102 optional ChangeAggregatorSpec changeAggregator = 25; 103 optional ChangeFrontierSpec changeFrontier = 26; 104 optional OrdinalitySpec ordinality = 27; 105 optional BulkRowWriterSpec bulkRowWriter = 28; 106 optional InvertedFiltererSpec invertedFilterer = 29; 107 108 reserved 6, 12; 109 } 110 111 // NoopCoreSpec indicates a "no-op" processor core. This is used when we just 112 // need post-processing or when only a synchronizer is required (e.g. at the 113 // final endpoint). 114 message NoopCoreSpec { 115 } 116 117 // LocalPlanNodeSpec is the specification for a local planNode wrapping 118 // processor. It's created for situations where a planNode has no DistSQL 119 // processor equivalent, and constrains the plan to live on the gateway node. 120 // This spec contains just an index, which is used by the execution engine to 121 // find the planNode instance this processor is executing in an array of local 122 // planNodes. See LocalProcessors and LocalProcessorIndexes on 123 // distsqlplan.PhysicalPlan. 124 message LocalPlanNodeSpec { 125 optional uint32 RowSourceIdx = 1; 126 optional uint32 NumInputs = 2; 127 optional string Name = 3; 128 } 129 130 message MetadataTestSenderSpec { 131 optional string id = 1 [(gogoproto.nullable) = false, 132 (gogoproto.customname) = "ID"]; 133 } 134 135 message MetadataTestReceiverSpec { 136 repeated string sender_ids = 1 [(gogoproto.customname) = "SenderIDs"]; 137 } 138