github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/execinfrapb/processors_bulk_io.proto (about) 1 // Copyright 2019 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 "jobs/jobspb/jobs.proto"; 23 import "roachpb/io-formats.proto"; 24 import "sql/sqlbase/structured.proto"; 25 import "sql/execinfrapb/processors_base.proto"; 26 import "util/hlc/timestamp.proto"; 27 import "gogoproto/gogo.proto"; 28 29 // BackfillerSpec is the specification for a "schema change backfiller". 30 // The created backfill processor runs a backfill for the first mutations in 31 // the table descriptor mutation list with the same mutation id and type. 32 // A backfiller processor performs KV operations to retrieve rows for a 33 // table and backfills the new indexes/columns contained in the table 34 // descriptor. It checkpoints its progress by updating the table 35 // descriptor in the database, and doesn't emit any rows nor support 36 // any post-processing. 37 message BackfillerSpec { 38 enum Type { 39 Invalid = 0; 40 Column = 1; 41 Index = 2; 42 } 43 optional Type type = 1 [(gogoproto.nullable) = false]; 44 optional sqlbase.TableDescriptor table = 2 [(gogoproto.nullable) = false]; 45 46 // Sections of the table to be backfilled. 47 repeated TableReaderSpan spans = 3 [(gogoproto.nullable) = false]; 48 49 // Run the backfill for approximately this duration. 50 // The backfill will always process at least one backfill chunk. 51 optional int64 duration = 4 [(gogoproto.nullable) = false, (gogoproto.casttype) = "time.Duration"]; 52 53 // The backfill involves a complete table scan in chunks, 54 // where each chunk is a transactional read of a set of rows 55 // along with a backfill for the rows. This is the maximum number 56 // of entries backfilled per chunk. 57 optional int64 chunk_size = 5 [(gogoproto.nullable) = false]; 58 59 // Any other (leased) table descriptors necessary for the 60 // backfiller to do its job, such as the descriptors for tables with fk 61 // relationships to the table being modified. 62 repeated sqlbase.TableDescriptor other_tables = 6 [(gogoproto.nullable) = false]; 63 64 // The timestamp to perform index backfill historical scans at. 65 optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false]; 66 } 67 68 // JobProgress identifies the job to report progress on. This reporting 69 // happens outside this package. 70 message JobProgress { 71 optional int64 job_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "JobID"]; 72 // contribution is the percent of work of the total this processor will 73 // process. 74 optional float contribution = 2 [(gogoproto.nullable) = false]; 75 // slot is the index into the job details for this processor's completion. 76 optional int32 slot = 3 [(gogoproto.nullable) = false]; 77 } 78 79 message ReadImportDataSpec { 80 reserved 1; 81 optional roachpb.IOFileFormat format = 8 [(gogoproto.nullable) = false]; 82 // sample_size is the rate at which to output rows, based on an input row's size. 83 optional int32 sample_size = 2 [(gogoproto.nullable) = false]; 84 reserved 3; 85 86 message ImportTable { 87 optional sqlbase.TableDescriptor desc = 1 [(gogoproto.nullable) = true]; 88 // targetCols is used to store the target columns for each existing table 89 // being imported into. These are the columns for which the processor should 90 // read and emit data (ignoring data for any other tables or columns outside 91 // of the targetCols, that is present in the input). 92 repeated string targetCols = 2 [(gogoproto.nullable) = true]; 93 } 94 95 // tables supports input formats that can read multiple tables. If it is 96 // non-empty, the keys specify the names of tables for which the processor 97 // should read and emit data (ignoring data for any other tables that is 98 // present in the input). 99 // 100 // TODO(dt): If a key has a nil value, the schema for that table should be 101 // determined from the input on-the-fly (e.g. by parsing a CREATE TABLE in a 102 // dump file) and the processor should emit a key/value for the generated 103 // TableDescriptor with the corresponding descriptor ID key. If tables is 104 // empty (and table_desc above is not specified), the processor should read 105 // all tables in the input, determining their schemas on the fly. 106 map<string, ImportTable> tables = 9 [(gogoproto.nullable) = true]; 107 108 // uri is a cloud.ExternalStorage URI pointing to the CSV files to be 109 // read. The map key must be unique across the entire IMPORT job. 110 map<int32, string> uri = 7; 111 112 // resume_pos specifies a map from an input ID to an offset in that 113 // input from which the processing should continue. 114 // The meaning of offset is specific to each processor. 115 map<int32, int64> resume_pos = 14; 116 117 optional JobProgress progress = 6 [(gogoproto.nullable) = false]; 118 119 reserved 4; 120 reserved 5; 121 122 optional bool skip_missing_foreign_keys = 10 [(gogoproto.nullable) = false]; 123 124 // walltimeNanos is the MVCC time at which the created KVs will be written. 125 optional int64 walltimeNanos = 11 [(gogoproto.nullable) = false]; 126 127 reserved 12; 128 129 // If set, specifies reader parallelism; 0 implies "use default". 130 optional int32 readerParallelism = 13 [(gogoproto.nullable) = false]; 131 132 // NEXTID: 14 133 } 134 135 // FileCompression list of the compression codecs which are currently 136 // supported for CSVWriter spec 137 enum FileCompression { 138 None = 0; 139 Gzip = 1; 140 } 141 142 // CSVWriterSpec is the specification for a processor that consumes rows and 143 // writes them to CSV files at uri. It outputs a row per file written with 144 // the file name, row count and byte size. 145 message CSVWriterSpec { 146 // destination as a cloud.ExternalStorage URI pointing to an export store 147 // location (directory). 148 optional string destination = 1 [(gogoproto.nullable) = false]; 149 optional string name_pattern = 2 [(gogoproto.nullable) = false]; 150 optional roachpb.CSVOptions options = 3 [(gogoproto.nullable) = false]; 151 // chunk_rows is num rows to write per file. 0 = no limit. 152 optional int64 chunk_rows = 4 [(gogoproto.nullable) = false]; 153 154 // compression_codec specifies compression used for exported file. 155 optional FileCompression compression_codec = 5 [(gogoproto.nullable) = false]; 156 } 157 158 // BulkRowWriterSpec is the specification for a processor that consumes rows and 159 // writes them to a target table using AddSSTable. It outputs a BulkOpSummary. 160 message BulkRowWriterSpec { 161 optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; 162 }