From 842cb0f25880dd4869aace189c8d65bb886f364a Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Mon, 22 Jun 2020 17:40:06 -0400 Subject: [PATCH] sql: expose mvcc timestamps to SQL Fixes #50102. This PR adds introspection into the KV layer's concurrency control from the SQL level. In particular, we now expose the MVCC HLC timestamp of a row as a special system column on every table. This system column is exposed as a decimal, and is computed as `wall time * 10^10 + logical time`. To accomplish this, this PR adds planning and execution infrastructure for implicit system columns that can be produced by the execution layer for a particular row. Release note (sql change): Expose the MVCC timestamp of each row as a system column on tables. This column is named `crdb_internal_mvcc_timestamp` and is accessible only in a limited set of contexts. --- pkg/ccl/backupccl/backup_test.go | 4 +- pkg/ccl/changefeedccl/encoder.go | 2 +- .../testdata/logic_test/partitioning | 12 + pkg/ccl/logictestccl/testdata/logic_test/zone | 6 + pkg/sql/colexec/colbuilder/execplan.go | 3 +- pkg/sql/colfetcher/cfetcher.go | 86 ++- pkg/sql/colfetcher/colbatch_scan.go | 50 +- pkg/sql/crdb_internal.go | 4 +- pkg/sql/distsql_physical_planner.go | 72 +- pkg/sql/distsql_spec_exec_factory.go | 32 +- pkg/sql/exec_factory_util.go | 14 + pkg/sql/execinfrapb/processors_sql.pb.go | 572 ++++++++++----- pkg/sql/execinfrapb/processors_sql.proto | 14 + .../testdata/logic_test/dist_vectorize | 30 +- .../experimental_distsql_planning_5node | 31 +- pkg/sql/logictest/testdata/logic_test/mvcc | 124 ++++ .../logictest/testdata/logic_test/tpch_vec | 129 ++-- .../logictest/testdata/logic_test/vectorize | 19 + .../testdata/logic_test/vectorize_agg | 4 +- .../testdata/logic_test/vectorize_local | 26 +- .../testdata/logic_test/vectorize_overloads | 64 +- pkg/sql/opt/cat/column.go | 11 + pkg/sql/opt/cat/index.go | 9 +- pkg/sql/opt/cat/utils.go | 9 +- pkg/sql/opt/exec/execbuilder/mutation.go | 3 +- .../opt/exec/execbuilder/testdata/aggregate | 122 +-- pkg/sql/opt/exec/execbuilder/testdata/catalog | 11 + .../opt/exec/execbuilder/testdata/distsql_agg | 8 +- pkg/sql/opt/exec/execbuilder/testdata/explain | 61 +- .../execbuilder/testdata/hash_sharded_index | 18 +- pkg/sql/opt/exec/execbuilder/testdata/insert | 102 +-- pkg/sql/opt/exec/execbuilder/testdata/join | 20 +- .../opt/exec/execbuilder/testdata/lookup_join | 6 +- pkg/sql/opt/exec/execbuilder/testdata/mvcc | 58 ++ pkg/sql/opt/exec/execbuilder/testdata/orderby | 8 +- .../execbuilder/testdata/select_for_update | 82 +-- pkg/sql/opt/exec/execbuilder/testdata/srfs | 8 +- .../opt/exec/execbuilder/testdata/subquery | 8 +- pkg/sql/opt/exec/execbuilder/testdata/upsert | 106 +-- pkg/sql/opt/exec/execbuilder/testdata/with | 2 +- pkg/sql/opt/memo/check_expr.go | 3 + pkg/sql/opt/norm/prune_cols_funcs.go | 10 + pkg/sql/opt/optbuilder/fk_cascade.go | 7 +- pkg/sql/opt/optbuilder/insert.go | 27 +- pkg/sql/opt/optbuilder/mutation_builder.go | 18 +- pkg/sql/opt/testutils/testcat/create_table.go | 15 +- pkg/sql/opt/testutils/testcat/test_catalog.go | 8 + pkg/sql/opt/xform/coster.go | 2 +- pkg/sql/opt_catalog.go | 48 +- pkg/sql/opt_exec_factory.go | 6 + pkg/sql/row/fetcher.go | 39 +- pkg/sql/row/fetcher_mvcc_test.go | 2 +- pkg/sql/row/kv_fetcher.go | 25 +- pkg/sql/rowexec/indexjoiner.go | 20 +- pkg/sql/rowexec/inverted_joiner.go | 2 +- pkg/sql/rowexec/joinreader.go | 2 +- pkg/sql/rowexec/rowfetcher.go | 4 + pkg/sql/rowexec/scrub_tablereader.go | 2 +- pkg/sql/rowexec/tablereader.go | 30 +- pkg/sql/rowexec/zigzagjoiner.go | 1 + pkg/sql/scan.go | 28 +- pkg/sql/sem/tree/eval.go | 19 +- pkg/sql/sqlbase/index_encoding.go | 26 +- pkg/sql/sqlbase/structured.go | 4 + pkg/sql/sqlbase/structured.pb.go | 694 ++++++++++-------- pkg/sql/sqlbase/structured.proto | 16 +- pkg/sql/sqlbase/system_columns.go | 137 ++++ pkg/sql/sqlbase/validate_test.go | 1 + pkg/sql/testdata/explain_tree | 8 +- 69 files changed, 2063 insertions(+), 1091 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/mvcc create mode 100644 pkg/sql/opt/exec/execbuilder/testdata/mvcc create mode 100644 pkg/sql/sqlbase/system_columns.go diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index ffec53b3a7fd..846e784b2b5d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -2660,7 +2660,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitNone) defer cleanupFn() const dir = "nodelocal://0/" - preGC := tree.TimestampToDecimal(tc.Server(0).Clock().Now()).String() + preGC := tree.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() gcr := roachpb.GCRequest{ // Bogus span to make it a valid request. @@ -2676,7 +2676,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { t.Fatal(err) } - postGC := tree.TimestampToDecimal(tc.Server(0).Clock().Now()).String() + postGC := tree.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() lateFullTableBackup := dir + "/tbl-after-gc" sqlDB.Exec(t, `BACKUP data.bank TO $1 WITH revision_history`, lateFullTableBackup) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index 80416ccf2f75..629842951628 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -251,7 +251,7 @@ func (e *jsonEncoder) EncodeResolvedTimestamp( _ context.Context, _ string, resolved hlc.Timestamp, ) ([]byte, error) { meta := map[string]interface{}{ - `resolved`: tree.TimestampToDecimal(resolved).Decimal.String(), + `resolved`: tree.TimestampToDecimalDatum(resolved).Decimal.String(), } var jsonEntries interface{} if e.wrapped { diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning b/pkg/ccl/logictestccl/testdata/logic_test/partitioning index 08ac596c9331..b49f81bc8e2a 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning @@ -423,6 +423,7 @@ TABLE ok1 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -465,6 +466,7 @@ TABLE ok2 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -507,6 +509,7 @@ TABLE ok3 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -552,6 +555,7 @@ TABLE ok4 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -590,6 +594,7 @@ TABLE ok5 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -653,6 +658,7 @@ TABLE ok6 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -688,6 +694,7 @@ TABLE ok7 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -729,6 +736,7 @@ TABLE ok8 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -772,6 +780,7 @@ TABLE ok9 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -817,6 +826,7 @@ TABLE ok10 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -871,6 +881,7 @@ TABLE ok11 ├── a int not null ├── b int not null ├── c int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -916,6 +927,7 @@ TABLE ok12 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null diff --git a/pkg/ccl/logictestccl/testdata/logic_test/zone b/pkg/ccl/logictestccl/testdata/logic_test/zone index b2915eaac4f6..878f3f87079d 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/zone +++ b/pkg/ccl/logictestccl/testdata/logic_test/zone @@ -39,6 +39,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -82,6 +83,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -145,6 +147,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -247,6 +250,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -300,6 +304,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -401,6 +406,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 27ff9047c68b..8474c057a99c 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -632,8 +632,7 @@ func NewColOperator( // still responsible for doing the cancellation check on their own while // performing long operations. result.Op = colexec.NewCancelChecker(result.Op) - returnMutations := core.TableReader.Visibility == execinfra.ScanVisibilityPublicAndNotPublic - result.ColumnTypes = core.TableReader.Table.ColumnTypesWithMutations(returnMutations) + result.ColumnTypes = scanOp.ResultTypes case core.Aggregator != nil: if err := checkNumIn(inputs, 1); err != nil { return r, err diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 433f3d87b2af..8fea03e16137 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -17,6 +17,7 @@ import ( "sort" "strings" + "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -32,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -93,6 +95,15 @@ type cTableInfo struct { // id pair at the start of the key. knownPrefixLength int + // The following fields contain MVCC metadata for each row and may be + // returned to users of cFetcher immediately after NextBatch returns. + // + // rowLastModified is the timestamp of the last time any family in the row + // was modified in any way. + rowLastModified hlc.Timestamp + // timestampOutputIdx controls at what row ordinal to write the timestamp. + timestampOutputIdx int + keyValTypes []*types.T extraTypes []*types.T @@ -143,6 +154,10 @@ func (m colIdxMap) get(c sqlbase.ColumnID) (int, bool) { return 0, false } +// noTimestampColumn is a sentinel value to denote that the MVCC timestamp +// column is not part of the output. +const noTimestampColumn = -1 + // cFetcher handles fetching kvs and forming table rows for an // arbitrary number of tables. // Usage: @@ -186,6 +201,11 @@ type cFetcher struct { // when beginning a new scan. traceKV bool + // mvccDecodeStrategy controls whether or not MVCC timestamps should + // be decoded from KV's fetched. It is set if any of the requested tables + // are required to produce an MVCC timestamp system column. + mvccDecodeStrategy row.MVCCDecodingStrategy + // fetcher is the underlying fetcher that provides KVs. fetcher *row.KVFetcher @@ -221,6 +241,11 @@ type cFetcher struct { // colvecs is a slice of the ColVecs within batch, pulled out to avoid // having to call batch.Vec too often in the tight loop. colvecs []coldata.Vec + + // timestampCol is the underlying ColVec for the timestamp output column, + // or nil if the timestamp column was not requested. It is pulled out from + // colvecs to avoid having to cast the vec to decimal on every write. + timestampCol []apd.Decimal } // adapter is a utility struct that helps with memory accounting. @@ -267,12 +292,13 @@ func (rf *cFetcher) Init( sort.Sort(m) colDescriptors := tableArgs.Cols table := &cTableInfo{ - spans: tableArgs.Spans, - desc: tableArgs.Desc, - colIdxMap: m, - index: tableArgs.Index, - isSecondaryIndex: tableArgs.IsSecondaryIndex, - cols: colDescriptors, + spans: tableArgs.Spans, + desc: tableArgs.Desc, + colIdxMap: m, + index: tableArgs.Index, + isSecondaryIndex: tableArgs.IsSecondaryIndex, + cols: colDescriptors, + timestampOutputIdx: noTimestampColumn, } typs := make([]*types.T, len(colDescriptors)) @@ -280,9 +306,6 @@ func (rf *cFetcher) Init( typs[i] = colDescriptors[i].Type } - rf.machine.batch = allocator.NewMemBatch(typs) - rf.machine.colvecs = rf.machine.batch.ColVecs() - var err error var neededCols util.FastIntSet @@ -294,10 +317,24 @@ func (rf *cFetcher) Init( // The idx-th column is required. neededCols.Add(int(col)) table.neededColsList = append(table.neededColsList, int(col)) + // If this column is the timestamp column, set up the output index. + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(col) + if sysColKind == sqlbase.SystemColumnKind_MVCCTIMESTAMP { + table.timestampOutputIdx = idx + rf.mvccDecodeStrategy = row.MVCCDecodingRequired + } } } sort.Ints(table.neededColsList) + rf.machine.batch = allocator.NewMemBatch(typs) + rf.machine.colvecs = rf.machine.batch.ColVecs() + // If the fetcher is requested to produce a timestamp column, pull out the + // column as a decimal and save it. + if table.timestampOutputIdx != noTimestampColumn { + rf.machine.timestampCol = rf.machine.colvecs[table.timestampOutputIdx].Decimal() + } + table.knownPrefixLength = len(sqlbase.MakeIndexKeyPrefix(codec, table.desc.TableDesc(), table.index.ID)) var indexColumnIDs []sqlbase.ColumnID @@ -309,6 +346,15 @@ func (rf *cFetcher) Init( } table.neededValueColsByIdx = tableArgs.ValNeededForCol.Copy() + + // If system columns are requested, they are present in ValNeededForCol. + // However, we don't want to include them in neededValueColsByIdx, because + // the handling of system columns is separate from the standard value + // decoding process. + if table.timestampOutputIdx != noTimestampColumn { + table.neededValueColsByIdx.Remove(table.timestampOutputIdx) + } + neededIndexCols := 0 nIndexCols := len(indexColumnIDs) if cap(table.indexColOrdinals) >= nIndexCols { @@ -590,7 +636,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { case stateInvalid: return nil, errors.New("invalid fetcher state") case stateInitFetch: - moreKeys, kv, newSpan, err := rf.fetcher.NextKV(ctx) + moreKeys, kv, newSpan, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -628,6 +674,9 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.machine.batch.ResetInternalBatch() rf.shiftState() case stateDecodeFirstKVOfRow: + // Reset MVCC metadata for the table, since this is the first KV of a row. + rf.table.rowLastModified = hlc.Timestamp{} + // foundNull is set when decoding a new index key for a row finds a NULL value // in the index key. This is used when decoding unique secondary indexes in order // to tell whether they have extra columns appended to the key. @@ -732,6 +781,10 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { if rf.traceKV { log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal) } + // Update the MVCC values for this row. + if rf.table.rowLastModified.Less(rf.machine.nextKV.Value.Timestamp) { + rf.table.rowLastModified = rf.machine.nextKV.Value.Timestamp + } if len(rf.table.desc.Families) == 1 { rf.machine.state[0] = stateFinalizeRow rf.machine.state[1] = stateInitFetch @@ -740,7 +793,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.machine.state[0] = stateFetchNextKVWithUnfinishedRow case stateSeekPrefix: for { - moreRows, kv, _, err := rf.fetcher.NextKV(ctx) + moreRows, kv, _, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -772,7 +825,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.shiftState() case stateFetchNextKVWithUnfinishedRow: - moreKVs, kv, _, err := rf.fetcher.NextKV(ctx) + moreKVs, kv, _, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -823,6 +876,11 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal) } + // Update the MVCC values for this row. + if rf.table.rowLastModified.Less(rf.machine.nextKV.Value.Timestamp) { + rf.table.rowLastModified = rf.machine.nextKV.Value.Timestamp + } + if familyID == rf.table.maxColumnFamilyID { // We know the row can't have any more keys, so finalize the row. rf.machine.state[0] = stateFinalizeRow @@ -833,6 +891,10 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { } case stateFinalizeRow: + // Populate the row with the buffered MVCC information. + if rf.table.timestampOutputIdx != noTimestampColumn { + rf.machine.timestampCol[rf.machine.rowIdx] = tree.TimestampToDecimal(rf.table.rowLastModified) + } // We're finished with a row. Bump the row index, fill the row in with // nulls if necessary, emit the batch if necessary, and move to the next // state. diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 4d29f4541f30..2b4ad10e8bc4 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -23,11 +23,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) -// TODO(yuzefovich): reading the data through a pair of colBatchScan and +// TODO(yuzefovich): reading the data through a pair of ColBatchScan and // materializer turns out to be more efficient than through a table reader (at // the moment, the exception is the case of reading very small number of rows // because we still pre-allocate batches of 1024 size). Once we can control the @@ -35,9 +36,9 @@ import ( // should get rid off table readers entirely. We will have to be careful about // propagating the metadata though. -// colBatchScan is the exec.Operator implementation of TableReader. It reads a table +// ColBatchScan is the exec.Operator implementation of TableReader. It reads a table // from kv, presenting it as coldata.Batches via the exec.Operator interface. -type colBatchScan struct { +type ColBatchScan struct { colexecbase.ZeroInputNode spans roachpb.Spans flowCtx *execinfra.FlowCtx @@ -47,11 +48,16 @@ type colBatchScan struct { ctx context.Context // init is true after Init() has been called. init bool + // ResultTypes is the slice of resulting column types from this operator. + // It should be used rather than the slice of column types from the scanned + // table because the scan might synthesize additional implicit system columns. + ResultTypes []*types.T } -var _ colexecbase.Operator = &colBatchScan{} +var _ colexecbase.Operator = &ColBatchScan{} -func (s *colBatchScan) Init() { +// Init initializes a ColBatchScan. +func (s *ColBatchScan) Init() { s.ctx = context.Background() s.init = true @@ -64,7 +70,8 @@ func (s *colBatchScan) Init() { } } -func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { +// Next is part of the Operator interface. +func (s *ColBatchScan) Next(ctx context.Context) coldata.Batch { bat, err := s.rf.NextBatch(ctx) if err != nil { colexecerror.InternalError(err) @@ -76,7 +83,7 @@ func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { } // DrainMeta is part of the MetadataSource interface. -func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { +func (s *ColBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { if !s.init { // In some pathological queries like `SELECT 1 FROM t HAVING true`, Init() // and Next() may never get called. Return early to avoid using an @@ -99,26 +106,37 @@ func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMeta return trailingMeta } -// NewColBatchScan creates a new colBatchScan operator. +// NewColBatchScan creates a new ColBatchScan operator. func NewColBatchScan( allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, -) (colexecbase.DrainableOperator, error) { +) (*ColBatchScan, error) { // NB: we hit this with a zero NodeID (but !ok) with multi-tenancy. if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); nodeID == 0 && ok { - return nil, errors.Errorf("attempting to create a colBatchScan with uninitialized NodeID") + return nil, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID") } limitHint := execinfra.LimitHint(spec.LimitHint, post) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic typs := spec.Table.ColumnTypesWithMutations(returnMutations) + columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&spec.Table, spec.SystemColumns) + if err != nil { + return nil, err + } + typs = append(typs, sysColTypes...) + for i := range sysColDescs { + columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + } + evalCtx := flowCtx.NewEvalCtx() // Before we can safely use types from the table descriptor, we need to // make sure they are hydrated. In row execution engine it is done during - // the processor initialization, but neither colBatchScan nor cFetcher are + // the processor initialization, but neither ColBatchScan nor cFetcher are // processors, so we need to do the hydration ourselves. if err := execinfrapb.HydrateTypeSlice(evalCtx, typs); err != nil { return nil, err @@ -135,7 +153,6 @@ func NewColBatchScan( neededColumns := helper.NeededColumns() - columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) fetcher := cFetcher{} if spec.IsCheck { // cFetchers don't support these checks. @@ -143,7 +160,7 @@ func NewColBatchScan( } if _, _, err := initCRowFetcher( flowCtx.Codec(), allocator, &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, - spec.Reverse, neededColumns, spec.Visibility, spec.LockingStrength, + spec.Reverse, neededColumns, spec.Visibility, spec.LockingStrength, sysColDescs, ); err != nil { return nil, err } @@ -153,7 +170,7 @@ func NewColBatchScan( for i := range spans { spans[i] = spec.Spans[i].Span } - return &colBatchScan{ + return &ColBatchScan{ spans: spans, flowCtx: flowCtx, rf: &fetcher, @@ -161,6 +178,7 @@ func NewColBatchScan( // Parallelize shouldn't be set when there's a limit hint, but double-check // just in case. parallelize: spec.Parallelize && limitHint == 0, + ResultTypes: typs, }, nil } @@ -176,6 +194,7 @@ func initCRowFetcher( valNeededForCol util.FastIntSet, scanVisibility execinfrapb.ScanVisibility, lockStr sqlbase.ScanLockingStrength, + systemColumnDescs []sqlbase.ColumnDescriptor, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -187,6 +206,9 @@ func initCRowFetcher( if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { cols = immutDesc.ReadableColumns } + // Add on any requested system columns. We slice cols to avoid modifying + // the underlying table descriptor. + cols = append(cols[:len(cols):len(cols)], systemColumnDescs...) tableArgs := row.FetcherTableArgs{ Desc: immutDesc, Index: index, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 922b5443b4a5..2259124c6c9e 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -285,7 +285,7 @@ CREATE TABLE crdb_internal.tables ( dbName, tree.NewDInt(tree.DInt(int64(table.Version))), tree.TimestampToInexactDTimestamp(table.ModificationTime), - tree.TimestampToDecimal(table.ModificationTime), + tree.TimestampToDecimalDatum(table.ModificationTime), tree.NewDString(table.FormatVersion.String()), tree.NewDString(table.State.String()), leaseNodeDatum, @@ -577,7 +577,7 @@ CREATE TABLE crdb_internal.jobs ( // Progress contains either fractionCompleted for traditional jobs, // or the highWaterTimestamp for change feeds. if highwater := progress.GetHighWater(); highwater != nil { - highWaterTimestamp = tree.TimestampToDecimal(*highwater) + highWaterTimestamp = tree.TimestampToDecimalDatum(*highwater) } else { fractionCompleted = tree.NewDFloat(tree.DFloat(progress.GetFractionCompleted())) } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 6a97e9b31ea1..c51ae3683c52 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -944,7 +944,8 @@ func initTableReaderSpec( LockingWaitPolicy: n.lockingWaitPolicy, // Retain the capacity of the spans slice. - Spans: s.Spans[:0], + Spans: s.Spans[:0], + SystemColumns: n.systemColumns, } indexIdx, err := getIndexIdx(n.index, n.desc) if err != nil { @@ -994,6 +995,13 @@ func tableOrdinal( } } } + + // The column is an implicit system column, so give it an ordinal based + // on its ID that is larger than physical columns. + if sqlbase.IsColIDSystemColumn(colID) { + return len(desc.Columns) + len(desc.MutationColumns()) + int(colID-sqlbase.MVCCTimestampColumnID) + } + panic(fmt.Sprintf("column %d not in desc.Columns", colID)) } @@ -1136,17 +1144,19 @@ func (dsp *DistSQLPlanner) createTableReaders( planCtx, &p, &tableReaderPlanningInfo{ - spec: spec, - post: post, - desc: n.desc, - spans: n.spans, - reverse: n.reverse, - scanVisibility: n.colCfg.visibility, - parallelize: n.parallelize, - estimatedRowCount: n.estimatedRowCount, - reqOrdering: n.reqOrdering, - cols: n.cols, - colsToTableOrdrinalMap: scanNodeToTableOrdinalMap, + spec: spec, + post: post, + desc: n.desc, + spans: n.spans, + reverse: n.reverse, + scanVisibility: n.colCfg.visibility, + parallelize: n.parallelize, + estimatedRowCount: n.estimatedRowCount, + reqOrdering: n.reqOrdering, + cols: n.cols, + colsToTableOrdinalMap: scanNodeToTableOrdinalMap, + systemColumns: n.systemColumns, + systemColumnOrdinals: n.systemColumnOrdinals, }, ) return &p, err @@ -1156,17 +1166,19 @@ func (dsp *DistSQLPlanner) createTableReaders( // needed to perform the physical planning of table readers once the specs have // been created. See scanNode to get more context on some of the fields. type tableReaderPlanningInfo struct { - spec *execinfrapb.TableReaderSpec - post execinfrapb.PostProcessSpec - desc *sqlbase.ImmutableTableDescriptor - spans []roachpb.Span - reverse bool - scanVisibility execinfrapb.ScanVisibility - parallelize bool - estimatedRowCount uint64 - reqOrdering ReqOrdering - cols []*sqlbase.ColumnDescriptor - colsToTableOrdrinalMap []int + spec *execinfrapb.TableReaderSpec + post execinfrapb.PostProcessSpec + desc *sqlbase.ImmutableTableDescriptor + spans []roachpb.Span + reverse bool + scanVisibility execinfrapb.ScanVisibility + parallelize bool + estimatedRowCount uint64 + reqOrdering ReqOrdering + cols []*sqlbase.ColumnDescriptor + colsToTableOrdinalMap []int + systemColumns []sqlbase.SystemColumnKind + systemColumnOrdinals []int } func (dsp *DistSQLPlanner) planTableReaders( @@ -1243,12 +1255,16 @@ func (dsp *DistSQLPlanner) planTableReaders( typs = append(typs, col.Type) } } + // Append all system column types to the output. + for _, kind := range info.systemColumns { + typs = append(typs, sqlbase.GetSystemColumnTypeForKind(kind)) + } p.AddNoInputStage( - corePlacement, info.post, typs, dsp.convertOrdering(info.reqOrdering, info.colsToTableOrdrinalMap), + corePlacement, info.post, typs, dsp.convertOrdering(info.reqOrdering, info.colsToTableOrdinalMap), ) - outCols := getOutputColumnsFromColsForScan(info.cols, info.colsToTableOrdrinalMap) + outCols := getOutputColumnsFromColsForScan(info.cols, info.colsToTableOrdinalMap) planToStreamColMap := make([]int, len(info.cols)) descColumnIDs := make([]sqlbase.ColumnID, 0, len(info.desc.Columns)) for i := range info.desc.Columns { @@ -1259,6 +1275,11 @@ func (dsp *DistSQLPlanner) planTableReaders( descColumnIDs = append(descColumnIDs, c.ID) } } + // Add all system column IDs to the projection. + for _, ord := range info.systemColumnOrdinals { + descColumnIDs = append(descColumnIDs, info.cols[ord].ID) + } + for i := range planToStreamColMap { planToStreamColMap[i] = -1 for j, c := range outCols { @@ -1930,6 +1951,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( Visibility: n.table.colCfg.visibility, LockingStrength: n.table.lockingStrength, LockingWaitPolicy: n.table.lockingWaitPolicy, + SystemColumns: n.table.systemColumns, } filter, err := physicalplan.MakeExpression( diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 17298ddf614a..8fd780a67d8e 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -189,6 +189,10 @@ func (e *distSQLSpecExecFactory) ConstructScan( tabDesc := table.(*optTable).desc indexDesc := index.(*optIndex).desc colCfg := makeScanColumnsConfig(table, params.NeededCols) + + // Check if any system columns are requested, as they need special handling. + systemColumns, systemColumnOrdinals := collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + sb := span.MakeBuilder(e.planner.ExecCfg().Codec, tabDesc.TableDesc(), indexDesc) // Note that initColsForScan and setting ResultColumns below are equivalent @@ -238,7 +242,8 @@ func (e *distSQLSpecExecFactory) ConstructScan( IsCheck: false, Visibility: colCfg.visibility, // Retain the capacity of the spans slice. - Spans: trSpec.Spans[:0], + Spans: trSpec.Spans[:0], + SystemColumns: systemColumns, } trSpec.IndexIdx, err = getIndexIdx(indexDesc, tabDesc) if err != nil { @@ -270,17 +275,19 @@ func (e *distSQLSpecExecFactory) ConstructScan( e.getPlanCtx(recommendation), &p, &tableReaderPlanningInfo{ - spec: trSpec, - post: post, - desc: tabDesc, - spans: spans, - reverse: params.Reverse, - scanVisibility: colCfg.visibility, - parallelize: params.Parallelize, - estimatedRowCount: uint64(params.EstimatedRowCount), - reqOrdering: ReqOrdering(reqOrdering), - cols: cols, - colsToTableOrdrinalMap: colsToTableOrdinalMap, + spec: trSpec, + post: post, + desc: tabDesc, + spans: spans, + reverse: params.Reverse, + scanVisibility: colCfg.visibility, + parallelize: params.Parallelize, + estimatedRowCount: uint64(params.EstimatedRowCount), + reqOrdering: ReqOrdering(reqOrdering), + cols: cols, + colsToTableOrdinalMap: colsToTableOrdinalMap, + systemColumns: systemColumns, + systemColumnOrdinals: systemColumnOrdinals, }, ) @@ -628,6 +635,7 @@ func (e *distSQLSpecExecFactory) ConstructLookupJoin( onCond tree.TypedExpr, reqOrdering exec.OutputOrdering, ) (exec.Node, error) { + // TODO (rohany): Implement production of system columns by the underlying scan here. return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: lookup join") } diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go index 9b61f7dec4e0..2a3d3330ba72 100644 --- a/pkg/sql/exec_factory_util.go +++ b/pkg/sql/exec_factory_util.go @@ -367,3 +367,17 @@ func constructVirtualScan( } return n, nil } + +func collectSystemColumnsFromCfg( + colCfg *scanColumnsConfig, table *sqlbase.TableDescriptor, +) (systemColumns []sqlbase.SystemColumnKind, systemColumnOrdinals []int) { + for i, id := range colCfg.wantedColumns { + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(sqlbase.ColumnID(id)) + if sysColKind != sqlbase.SystemColumnKind_NONE { + // The scan is requested to produce a system column. + systemColumns = append(systemColumns, sysColKind) + systemColumnOrdinals = append(systemColumnOrdinals, i) + } + } + return systemColumns, systemColumnOrdinals +} diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 023f4c5f1115..693e9b943ba8 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -64,7 +64,7 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { return nil } func (ScanVisibility) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -181,7 +181,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{12, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{12, 0} } type AggregatorSpec_Type int32 @@ -227,7 +227,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{12, 1} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{12, 1} } type WindowerSpec_WindowFunc int32 @@ -291,7 +291,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 0} } // Mode indicates which mode of framing is used. @@ -335,7 +335,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1, 0} } // BoundType indicates which type of boundary is used. @@ -382,7 +382,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1, 1} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -425,7 +425,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1, 2} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -445,7 +445,7 @@ func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } func (*ValuesCoreSpec) ProtoMessage() {} func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,13 +546,17 @@ type TableReaderSpec struct { // makes it out of the SQL optimizer without throwing an error. If/when other // wait policies are supported, this field will be plumbed further. LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,11,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` + // Indicates what implicit system columns this TableReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the TableReader, in the order specified. + SystemColumns []sqlbase.SystemColumnKind `protobuf:"varint,13,rep,name=system_columns,json=systemColumns,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_columns,omitempty"` } func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } func (*TableReaderSpec) ProtoMessage() {} func (*TableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{1} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -610,7 +614,7 @@ func (m *IndexSkipTableReaderSpec) Reset() { *m = IndexSkipTableReaderSp func (m *IndexSkipTableReaderSpec) String() string { return proto.CompactTextString(m) } func (*IndexSkipTableReaderSpec) ProtoMessage() {} func (*IndexSkipTableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{2} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{2} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -706,13 +710,21 @@ type JoinReaderSpec struct { // stream. This is only applicable to lookup joins, where doing so is // expensive. Index joins do this by default regardless of the parameter. MaintainOrdering bool `protobuf:"varint,11,opt,name=maintain_ordering,json=maintainOrdering" json:"maintain_ordering"` + // Indicates what implicit system columns this JoinReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the JoinReader, in the order specified. + // + // This is only used in the special case of index joins, where the final + // result of the secondary index joined against the primary index is + // expected to contain the materialized system columns. + SystemColumns []sqlbase.SystemColumnKind `protobuf:"varint,12,rep,name=system_columns,json=systemColumns,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_columns,omitempty"` } func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } func (*JoinReaderSpec) ProtoMessage() {} func (*JoinReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{3} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{3} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -756,7 +768,7 @@ func (m *SorterSpec) Reset() { *m = SorterSpec{} } func (m *SorterSpec) String() string { return proto.CompactTextString(m) } func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{4} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{4} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -818,7 +830,7 @@ func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } func (*DistinctSpec) ProtoMessage() {} func (*DistinctSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{5} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{5} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -853,7 +865,7 @@ func (m *OrdinalitySpec) Reset() { *m = OrdinalitySpec{} } func (m *OrdinalitySpec) String() string { return proto.CompactTextString(m) } func (*OrdinalitySpec) ProtoMessage() {} func (*OrdinalitySpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{6} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{6} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -911,7 +923,7 @@ func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{7} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{7} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -987,7 +999,7 @@ func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{8} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{8} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1074,7 +1086,7 @@ func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} func (*HashJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{9} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{9} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1171,7 +1183,7 @@ func (m *InvertedJoinerSpec) Reset() { *m = InvertedJoinerSpec{} } func (m *InvertedJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InvertedJoinerSpec) ProtoMessage() {} func (*InvertedJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{10} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{10} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1224,7 +1236,7 @@ func (m *InvertedFiltererSpec) Reset() { *m = InvertedFiltererSpec{} } func (m *InvertedFiltererSpec) String() string { return proto.CompactTextString(m) } func (*InvertedFiltererSpec) ProtoMessage() {} func (*InvertedFiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{11} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{11} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1270,7 +1282,7 @@ func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} func (*AggregatorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{12} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{12} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1321,7 +1333,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{12, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{12, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1396,7 +1408,7 @@ func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJo func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{13} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{13} } func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1452,7 +1464,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedRe func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{13, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{13, 0} } func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1504,7 @@ func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{14} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1534,7 +1546,7 @@ func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} func (*WindowerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1570,7 +1582,7 @@ func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1606,7 +1618,7 @@ func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1647,7 +1659,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1, 0} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1683,7 +1695,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 1, 1} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1733,7 +1745,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_994fdcc7dfa95bad, []int{15, 2} + return fileDescriptor_processors_sql_9f5d15014b584a4d, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1908,6 +1920,13 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + if len(m.SystemColumns) > 0 { + for _, num := range m.SystemColumns { + dAtA[i] = 0x68 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) + } + } return i, nil } @@ -2048,6 +2067,13 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + if len(m.SystemColumns) > 0 { + for _, num := range m.SystemColumns { + dAtA[i] = 0x60 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) + } + } return i, nil } @@ -3026,6 +3052,11 @@ func (m *TableReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) n += 2 + if len(m.SystemColumns) > 0 { + for _, e := range m.SystemColumns { + n += 1 + sovProcessorsSql(uint64(e)) + } + } return n } @@ -3075,6 +3106,11 @@ func (m *JoinReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) n += 2 + if len(m.SystemColumns) > 0 { + for _, e := range m.SystemColumns { + n += 1 + sovProcessorsSql(uint64(e)) + } + } return n } @@ -3892,6 +3928,72 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } } m.Parallelize = bool(v != 0) + case 13: + if wireType == 0 { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessorsSql + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.SystemColumns) == 0 { + m.SystemColumns = make([]sqlbase.SystemColumnKind, 0, elementCount) + } + for iNdEx < postIndex { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -4417,6 +4519,72 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { } } m.MaintainOrdering = bool(v != 0) + case 12: + if wireType == 0 { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessorsSql + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.SystemColumns) == 0 { + m.SystemColumns = make([]sqlbase.SystemColumnKind, 0, elementCount) + } + for iNdEx < postIndex { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -7942,175 +8110,177 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_994fdcc7dfa95bad) -} - -var fileDescriptor_processors_sql_994fdcc7dfa95bad = []byte{ - // 2649 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0x4b, 0x73, 0x1b, 0xc7, - 0xf1, 0xe7, 0xe2, 0x41, 0x02, 0x8d, 0x07, 0x47, 0x23, 0xda, 0x82, 0x21, 0x17, 0x45, 0xc1, 0xfe, - 0xdb, 0x94, 0x2c, 0x93, 0x7f, 0x33, 0x29, 0xa7, 0x6c, 0xe7, 0x60, 0x3c, 0x16, 0x14, 0x28, 0x70, - 0x97, 0x5a, 0x00, 0x92, 0x1f, 0x55, 0xde, 0x5a, 0x62, 0x87, 0xe0, 0x5a, 0x8b, 0x5d, 0x70, 0x77, - 0x21, 0x92, 0xfe, 0x02, 0xb9, 0x26, 0x95, 0x4b, 0x4e, 0x29, 0xe7, 0x90, 0xf8, 0x0b, 0xe4, 0xec, - 0x5c, 0x72, 0xd0, 0xd1, 0xa7, 0xc4, 0xa7, 0x54, 0x2c, 0x7f, 0x81, 0xdc, 0x5c, 0xb9, 0xa5, 0xe6, - 0xb1, 0xcb, 0x05, 0x8a, 0x50, 0x04, 0x89, 0xb6, 0x2a, 0x17, 0x16, 0xa6, 0x7b, 0xfa, 0xd7, 0x3d, - 0xdd, 0x3d, 0x3d, 0x3d, 0xb3, 0x84, 0xd7, 0xfd, 0x23, 0x7b, 0x93, 0x9c, 0x90, 0xbe, 0xe5, 0x1c, - 0x78, 0xc6, 0x68, 0x7f, 0x73, 0xe4, 0xb9, 0x7d, 0xe2, 0xfb, 0xae, 0xe7, 0xeb, 0xfe, 0x91, 0xbd, - 0x31, 0xf2, 0xdc, 0xc0, 0xc5, 0xa5, 0xbe, 0xdb, 0x7f, 0xe0, 0xb9, 0x46, 0xff, 0x70, 0x83, 0x12, - 0x4d, 0xcb, 0x0f, 0xfc, 0x23, 0xdb, 0x1b, 0x3b, 0xe5, 0x57, 0xa9, 0xbc, 0x7f, 0x64, 0xef, 0x1b, - 0x3e, 0xd9, 0xf4, 0x03, 0x6f, 0xdc, 0x0f, 0xc6, 0x1e, 0x31, 0xb9, 0x5c, 0xf9, 0x6a, 0x9c, 0xfb, - 0xb9, 0x6b, 0x39, 0x7a, 0x70, 0x3a, 0x22, 0x82, 0xf9, 0x4a, 0x9c, 0x69, 0xbb, 0xfd, 0x07, 0x96, - 0x33, 0x10, 0xac, 0x97, 0x28, 0x8b, 0x4e, 0xf5, 0xf9, 0x5f, 0x41, 0x2e, 0x4f, 0x1b, 0x6b, 0x1a, - 0x81, 0x21, 0x78, 0xff, 0xf7, 0x84, 0x85, 0x50, 0x0d, 0x62, 0xda, 0x4d, 0x3a, 0xcd, 0x1d, 0x05, - 0x9b, 0x96, 0xf3, 0x90, 0x78, 0x01, 0x31, 0xc9, 0xc9, 0xc8, 0xdb, 0xf4, 0x47, 0x86, 0xa3, 0xd3, - 0x5f, 0xc4, 0xf7, 0x2d, 0xd7, 0x11, 0x73, 0x57, 0x06, 0xee, 0xc0, 0x65, 0x3f, 0x37, 0xe9, 0x2f, - 0x4e, 0xad, 0xfc, 0x56, 0x82, 0xe2, 0x3d, 0xc3, 0x1e, 0x13, 0xbf, 0xee, 0x7a, 0xa4, 0x33, 0x22, - 0x7d, 0x5c, 0x87, 0xa5, 0xbe, 0x6b, 0x8f, 0x87, 0x8e, 0x5f, 0x92, 0xd6, 0x92, 0xeb, 0xb9, 0xad, - 0xd7, 0x36, 0x66, 0x39, 0x6c, 0xa3, 0x61, 0x04, 0xe3, 0x61, 0xcb, 0x39, 0x70, 0x6b, 0xa9, 0x47, - 0xff, 0xb8, 0xb6, 0xa0, 0x85, 0x92, 0xf8, 0x2a, 0x64, 0x3d, 0xe3, 0x58, 0xdf, 0x3f, 0x0d, 0x88, - 0x5f, 0x4a, 0xac, 0x25, 0xd7, 0xf3, 0x5a, 0xc6, 0x33, 0x8e, 0x6b, 0x74, 0x8c, 0xaf, 0x41, 0xc6, - 0x19, 0x0f, 0x75, 0xcf, 0x3d, 0xf6, 0x4b, 0xc9, 0x35, 0x69, 0x3d, 0x15, 0x4a, 0x3b, 0xe3, 0xa1, - 0xe6, 0x1e, 0xfb, 0x95, 0x3f, 0xa4, 0x61, 0xb9, 0x6b, 0xec, 0xdb, 0x44, 0x23, 0x86, 0x49, 0x3c, - 0x66, 0x56, 0x0d, 0xd2, 0x01, 0x25, 0x95, 0xa4, 0x35, 0x69, 0x3d, 0xb7, 0xf5, 0xc6, 0x94, 0x51, - 0xc2, 0xf5, 0x1b, 0x4c, 0xac, 0x41, 0xfc, 0xbe, 0x67, 0x8d, 0x02, 0xd7, 0x13, 0xc8, 0x5c, 0x14, - 0x5f, 0x87, 0xac, 0xe5, 0x98, 0xe4, 0x44, 0xb7, 0xcc, 0x93, 0x52, 0x62, 0x4d, 0x5a, 0x2f, 0x08, - 0x7e, 0x86, 0x91, 0x5b, 0xe6, 0x09, 0x5e, 0x85, 0x25, 0x8f, 0x3c, 0x24, 0x9e, 0x4f, 0x98, 0x69, - 0x99, 0xd0, 0x34, 0x41, 0xc4, 0x32, 0xa4, 0xa9, 0x7f, 0xfd, 0x52, 0x8a, 0xf9, 0xe6, 0xc6, 0x6c, - 0xdf, 0x4c, 0x2c, 0xc0, 0x70, 0x42, 0x4b, 0x98, 0x34, 0x7e, 0x0d, 0xc0, 0xb6, 0x86, 0x56, 0xa0, - 0x1f, 0x5a, 0x4e, 0x50, 0x4a, 0xaf, 0x49, 0xeb, 0x49, 0x31, 0x21, 0xcb, 0xe8, 0xb7, 0x2d, 0x27, - 0xa0, 0x7e, 0xb2, 0x7c, 0xbd, 0x7f, 0x48, 0xfa, 0x0f, 0x4a, 0x8b, 0x71, 0x63, 0x2c, 0xbf, 0x4e, - 0x89, 0x58, 0x01, 0x78, 0x68, 0xf9, 0xd6, 0xbe, 0x65, 0x5b, 0xc1, 0x69, 0x69, 0x69, 0x4d, 0x5a, - 0x2f, 0x6e, 0xad, 0xcf, 0xb6, 0xa8, 0xd3, 0x37, 0x9c, 0x7b, 0xd1, 0x7c, 0x01, 0x16, 0x43, 0xc0, - 0x1f, 0xc0, 0x95, 0xa1, 0x71, 0xa2, 0x07, 0xd6, 0x90, 0xf8, 0x81, 0x31, 0x1c, 0xe9, 0xc6, 0x80, - 0xe8, 0x8e, 0xe1, 0xb8, 0x7e, 0x29, 0x1b, 0x8b, 0xd3, 0xca, 0xd0, 0x38, 0xe9, 0x86, 0x73, 0xaa, - 0x03, 0xa2, 0xd0, 0x19, 0xf8, 0x53, 0x40, 0x22, 0xef, 0x75, 0x3f, 0xf0, 0x88, 0x33, 0x08, 0x0e, - 0x4b, 0xc0, 0x4c, 0xba, 0x39, 0x23, 0x56, 0xd4, 0x9e, 0x36, 0x17, 0xe9, 0x08, 0x09, 0xa1, 0x61, - 0xd9, 0x9e, 0x24, 0xe3, 0x7d, 0xb8, 0x1c, 0x82, 0x1f, 0x1b, 0x56, 0xa0, 0x8f, 0x5c, 0xdb, 0xea, - 0x9f, 0x96, 0x72, 0x0c, 0xff, 0xd6, 0x7f, 0xc7, 0xbf, 0x6f, 0x58, 0xc1, 0x1e, 0x93, 0x11, 0x1a, - 0x2e, 0xd9, 0xd3, 0x0c, 0xfc, 0x06, 0xe4, 0x46, 0x86, 0x67, 0xd8, 0x36, 0xb1, 0xad, 0x2f, 0x48, - 0x29, 0x1f, 0xf3, 0x78, 0x9c, 0xb1, 0x93, 0xca, 0x64, 0x50, 0xb6, 0xf2, 0xef, 0x24, 0x94, 0x5a, - 0x34, 0x6b, 0x3a, 0x0f, 0xac, 0xd1, 0x0b, 0x4a, 0xd6, 0x28, 0x19, 0x93, 0xcf, 0x95, 0x8c, 0x93, - 0x69, 0x94, 0x7a, 0xee, 0x34, 0x8a, 0xed, 0xa1, 0xf4, 0x79, 0x7b, 0xe8, 0xbc, 0x4c, 0x59, 0xfc, - 0x91, 0x33, 0x65, 0xe9, 0x02, 0x33, 0xa5, 0xf2, 0x55, 0x1a, 0x8a, 0x3b, 0xae, 0xe5, 0xfc, 0xf4, - 0x11, 0xbf, 0x01, 0x45, 0xdb, 0x75, 0x1f, 0x8c, 0x47, 0x7a, 0x58, 0xa3, 0x69, 0xe8, 0x0b, 0xb5, - 0x04, 0x92, 0xb4, 0x02, 0xe7, 0xd4, 0x45, 0x09, 0xae, 0xc3, 0x92, 0xcb, 0xcf, 0x01, 0x16, 0xd2, - 0xdc, 0xd6, 0xeb, 0xb3, 0x43, 0x2a, 0x47, 0xa7, 0x85, 0xd0, 0xb8, 0xe8, 0x3a, 0x94, 0x86, 0xdf, - 0x83, 0x14, 0x3d, 0xb3, 0x44, 0x78, 0xae, 0xcd, 0x58, 0x15, 0xf5, 0x45, 0xf7, 0x74, 0x44, 0x84, - 0x30, 0x13, 0xb9, 0xf0, 0xe2, 0xf4, 0x1e, 0xbc, 0x3c, 0xb9, 0x74, 0xdd, 0xf0, 0x88, 0xfe, 0x80, - 0x9c, 0x96, 0x32, 0xb1, 0x24, 0xbb, 0x3c, 0xe1, 0x84, 0xaa, 0x47, 0xee, 0x90, 0xd3, 0x73, 0x13, - 0x2e, 0xfb, 0x23, 0x27, 0x1c, 0x5c, 0x64, 0x69, 0x7a, 0x07, 0x2e, 0x0d, 0x0d, 0xcb, 0x09, 0x0c, - 0xcb, 0xd1, 0x5d, 0xcf, 0x24, 0x9e, 0xe5, 0x0c, 0x58, 0xf1, 0x0b, 0x97, 0x8d, 0x42, 0xb6, 0x2a, - 0xb8, 0x3b, 0xa9, 0x4c, 0x1a, 0x2d, 0xd2, 0xf3, 0x1d, 0x3a, 0xae, 0x17, 0x88, 0x2c, 0xbd, 0x0b, - 0xcb, 0xee, 0x38, 0x18, 0x8d, 0x83, 0x33, 0x14, 0x9e, 0xaf, 0x95, 0xd9, 0x81, 0x09, 0x11, 0x85, - 0xa6, 0x22, 0x07, 0x08, 0xa9, 0x78, 0x0b, 0x70, 0x88, 0xa5, 0x0f, 0x8d, 0xa0, 0x7f, 0xa8, 0xdb, - 0xc4, 0x99, 0xc8, 0x5e, 0x14, 0xf2, 0x77, 0x29, 0xbb, 0x4d, 0x9c, 0xca, 0xd7, 0x12, 0xe4, 0x1b, - 0x96, 0x1f, 0x58, 0x4e, 0x3f, 0x60, 0x76, 0xbd, 0x09, 0xcb, 0x6c, 0x12, 0x31, 0xf5, 0x78, 0xef, - 0x51, 0xd0, 0x8a, 0x82, 0x1c, 0x26, 0xf5, 0x0d, 0x40, 0xa6, 0x10, 0x8c, 0x66, 0x26, 0xd8, 0xcc, - 0xe5, 0x90, 0x1e, 0x4e, 0xdd, 0x02, 0xec, 0x8c, 0x6d, 0x9b, 0xa7, 0x49, 0xc8, 0x9c, 0x38, 0xd4, - 0x11, 0xe3, 0x57, 0x3d, 0x12, 0xda, 0x82, 0xdf, 0x80, 0x3c, 0xf1, 0x3c, 0xd7, 0xd3, 0x5d, 0x47, - 0x37, 0xc7, 0x23, 0xb6, 0x71, 0xb2, 0x61, 0x2e, 0x32, 0x8e, 0xea, 0x34, 0xc6, 0xa3, 0x0a, 0x82, - 0xa2, 0xea, 0x99, 0x96, 0x63, 0xd0, 0xcc, 0xa4, 0x2b, 0xa8, 0xfc, 0x2e, 0x09, 0xe8, 0x13, 0x6b, - 0xf0, 0x85, 0x31, 0xa0, 0x9b, 0x41, 0xb8, 0xbb, 0x01, 0x8b, 0x6c, 0x67, 0x87, 0x9d, 0xd4, 0x7c, - 0x55, 0x41, 0xc8, 0xe2, 0x26, 0x00, 0x39, 0x9a, 0x58, 0x6d, 0x6e, 0xeb, 0xfa, 0xec, 0x78, 0x89, - 0xf5, 0x87, 0xed, 0x04, 0x39, 0x3a, 0xf3, 0x5d, 0x91, 0x97, 0x17, 0x97, 0x9b, 0x3e, 0x51, 0x3b, - 0x18, 0x47, 0xac, 0xe9, 0x82, 0x6a, 0xc7, 0x1d, 0xc8, 0x1f, 0x58, 0x27, 0xc4, 0xd4, 0x1f, 0xb2, - 0x06, 0xb3, 0x94, 0x66, 0x96, 0x3f, 0xa1, 0x04, 0x4c, 0x36, 0xa2, 0x5a, 0x8e, 0x49, 0x73, 0xe2, - 0x73, 0x14, 0xa2, 0xca, 0xdf, 0x92, 0xb0, 0xbc, 0x4b, 0xbc, 0x01, 0x89, 0x45, 0x66, 0x17, 0x0a, - 0x36, 0x39, 0x78, 0x8e, 0x6d, 0x90, 0xa7, 0xe2, 0xd1, 0x26, 0x50, 0xa1, 0xe8, 0x59, 0x83, 0xc3, - 0x18, 0x5e, 0x62, 0x4e, 0xbc, 0x02, 0x93, 0x8f, 0x00, 0x63, 0x01, 0x48, 0xbf, 0x88, 0xe2, 0x7d, - 0x03, 0x0a, 0x74, 0x73, 0xe8, 0xe4, 0x68, 0x6c, 0x44, 0xf5, 0x3b, 0xdc, 0x37, 0x79, 0xca, 0x92, - 0x05, 0x07, 0xbf, 0x0f, 0x57, 0x98, 0x2b, 0xcf, 0x72, 0x74, 0x46, 0x61, 0x26, 0x07, 0x81, 0x7c, - 0x34, 0x59, 0x98, 0x7f, 0x09, 0x25, 0xee, 0xb7, 0x73, 0x84, 0xb3, 0x31, 0xe1, 0x15, 0x36, 0x6b, - 0x4a, 0xba, 0xf2, 0x9b, 0x24, 0x14, 0x6f, 0x1b, 0xfe, 0x61, 0x2c, 0xae, 0x37, 0x61, 0x79, 0xca, - 0x18, 0x5e, 0x48, 0xc4, 0x01, 0x19, 0x37, 0x01, 0xdf, 0x02, 0x34, 0xad, 0x9c, 0xd7, 0x12, 0x36, - 0xb9, 0x38, 0xa9, 0xf2, 0x85, 0x47, 0xe4, 0x2d, 0x28, 0x0e, 0x69, 0x12, 0x9f, 0x55, 0xc8, 0x78, - 0x48, 0x0a, 0x9c, 0x17, 0x1a, 0xfb, 0xe2, 0x62, 0xf2, 0x43, 0x02, 0x70, 0x4b, 0xdc, 0x46, 0x63, - 0x71, 0xf9, 0xc9, 0xda, 0xa3, 0xc2, 0x44, 0x8f, 0xc0, 0xca, 0x7d, 0x21, 0xda, 0xb2, 0xb1, 0xd6, - 0x00, 0xab, 0x50, 0x08, 0x6f, 0xcd, 0xcf, 0x5a, 0xe8, 0xf2, 0x21, 0x00, 0x8b, 0xed, 0x0b, 0x4e, - 0x90, 0xca, 0x9f, 0x24, 0x58, 0x09, 0x3d, 0xdf, 0xb4, 0xec, 0x80, 0x78, 0xc2, 0xf7, 0x1b, 0x80, - 0xa2, 0x95, 0xf6, 0x5d, 0x9b, 0xb9, 0x4f, 0x8a, 0xf9, 0xa5, 0x18, 0x72, 0xeb, 0xae, 0x4d, 0x9d, - 0xf8, 0xd9, 0xb4, 0x67, 0x78, 0x2d, 0xfb, 0xd9, 0x94, 0x31, 0xee, 0x28, 0xd8, 0x88, 0xbf, 0x3b, - 0x6c, 0xd0, 0x8b, 0xc5, 0xd9, 0xc2, 0xf6, 0x3c, 0x37, 0x70, 0xcf, 0x73, 0x54, 0xe5, 0x5f, 0x4b, - 0x50, 0xac, 0x0e, 0x06, 0x1e, 0x19, 0x18, 0x81, 0xcb, 0x4d, 0xbc, 0x0e, 0x30, 0xf0, 0x5c, 0x1e, - 0xb6, 0xf8, 0x26, 0xcc, 0x32, 0x6a, 0xdd, 0xb5, 0x7d, 0xfc, 0x19, 0xe4, 0x0d, 0x21, 0x64, 0xb9, - 0xd1, 0x95, 0xe7, 0xe7, 0xb3, 0x7d, 0x3c, 0xa9, 0x22, 0x1a, 0xc6, 0xc2, 0x17, 0xc7, 0xc3, 0xff, - 0x2f, 0xfa, 0x18, 0x62, 0xea, 0x31, 0x53, 0x52, 0x91, 0x29, 0x48, 0x70, 0xb7, 0x23, 0x8b, 0xb6, - 0x45, 0xac, 0xd2, 0x2c, 0x56, 0x6f, 0x3f, 0xb5, 0x25, 0xd3, 0x91, 0x2b, 0xff, 0x2a, 0x01, 0xb9, - 0x98, 0x79, 0x14, 0xf8, 0x60, 0xec, 0xf4, 0x59, 0x90, 0xe6, 0x01, 0x6e, 0x8e, 0x9d, 0x7e, 0x08, - 0x4c, 0x01, 0xf0, 0x1a, 0x64, 0xa2, 0xc6, 0x27, 0x11, 0xdb, 0xba, 0x11, 0x15, 0xbf, 0x0e, 0xc5, - 0x03, 0x96, 0x2b, 0x51, 0x66, 0xd0, 0x6d, 0x50, 0xd0, 0xf2, 0x9c, 0x2a, 0x32, 0xe2, 0x0a, 0x7b, - 0x12, 0x62, 0xec, 0x34, 0x6b, 0xb6, 0x16, 0xfb, 0x9c, 0x71, 0x1b, 0xb2, 0x86, 0x37, 0x18, 0x0f, - 0x89, 0x13, 0xf8, 0xa5, 0x45, 0x16, 0x91, 0x79, 0xb2, 0xfe, 0x4c, 0x78, 0x27, 0x95, 0x49, 0xa2, - 0x54, 0xe5, 0xab, 0x24, 0xa4, 0xe8, 0x2a, 0x30, 0x82, 0x7c, 0x55, 0xf9, 0x58, 0x57, 0xd4, 0xae, - 0xae, 0xf4, 0xda, 0x6d, 0xb4, 0x80, 0x97, 0x20, 0x59, 0xbd, 0xb7, 0x8d, 0x24, 0x9c, 0x87, 0x4c, - 0x4d, 0x55, 0xdb, 0x7a, 0x55, 0x69, 0xa0, 0x04, 0xce, 0xc1, 0x12, 0x1b, 0xa9, 0x1a, 0x4a, 0xe2, - 0x22, 0x40, 0x5d, 0x55, 0xea, 0xd5, 0xae, 0x5e, 0xdd, 0xde, 0x46, 0x29, 0x9c, 0x85, 0x74, 0x5d, - 0xed, 0x29, 0x5d, 0x94, 0xa6, 0xe2, 0xbb, 0xd5, 0x8f, 0xd0, 0x12, 0xfb, 0xd1, 0x52, 0x50, 0x06, - 0x03, 0x2c, 0x76, 0xba, 0x8d, 0x86, 0x7c, 0x0f, 0x65, 0x29, 0xb1, 0xd3, 0xdb, 0x45, 0x40, 0xe1, - 0x3a, 0xbd, 0x5d, 0xbd, 0xa5, 0x74, 0x51, 0x8e, 0x6a, 0xba, 0x57, 0xd5, 0x5a, 0x55, 0xa5, 0x2e, - 0xa3, 0x3c, 0x65, 0x7d, 0xa4, 0x6a, 0x0c, 0xb9, 0xc0, 0x35, 0xf5, 0x94, 0xae, 0xae, 0xa9, 0xf7, - 0x3b, 0xa8, 0xc8, 0xe4, 0xee, 0x6a, 0x8d, 0x56, 0xb3, 0x89, 0x96, 0x31, 0x86, 0x62, 0xb3, 0xa5, - 0x54, 0xdb, 0x7a, 0x24, 0x8d, 0xe8, 0x82, 0x38, 0x4d, 0xe8, 0xbc, 0x84, 0x0b, 0x90, 0xad, 0x6a, - 0x5a, 0xf5, 0x63, 0x86, 0x88, 0xa9, 0xb2, 0x9d, 0x8e, 0xaa, 0xb0, 0xd1, 0x65, 0xca, 0xa4, 0xa3, - 0x1a, 0x1b, 0xae, 0x50, 0x75, 0x9d, 0xae, 0xd6, 0x52, 0xb6, 0xd9, 0xf8, 0x25, 0xb6, 0xea, 0x56, - 0x97, 0xb9, 0xe0, 0x65, 0xba, 0x10, 0x3a, 0x50, 0x35, 0x74, 0x05, 0x67, 0x20, 0x55, 0x57, 0x35, - 0x0d, 0x95, 0x70, 0x09, 0x56, 0xf6, 0x64, 0xad, 0x2e, 0x2b, 0xdd, 0x56, 0x5b, 0xd6, 0x1b, 0xad, - 0x4e, 0x5d, 0x6f, 0xed, 0xee, 0xb5, 0xd1, 0x2b, 0x53, 0x9c, 0xba, 0xaa, 0x74, 0x39, 0xa7, 0x8c, - 0x2f, 0xc3, 0x32, 0xb3, 0x41, 0xad, 0xed, 0xc8, 0x75, 0xee, 0xc4, 0xab, 0x78, 0x05, 0x10, 0x37, - 0x25, 0x46, 0x7d, 0xb5, 0x72, 0x0b, 0x52, 0x34, 0x8f, 0xa9, 0xc2, 0x6a, 0xaf, 0xab, 0xa2, 0x05, - 0xe6, 0xcf, 0x7a, 0xb5, 0x5d, 0xd5, 0x90, 0x44, 0xed, 0x55, 0x54, 0x45, 0x17, 0xe3, 0x44, 0xe5, - 0x2f, 0x8b, 0x70, 0xb5, 0xe5, 0x04, 0xc4, 0xb3, 0x89, 0xf1, 0x90, 0x98, 0xfc, 0xde, 0x1c, 0x3b, - 0x1e, 0x3e, 0x9e, 0x6a, 0x94, 0x3f, 0x98, 0x9d, 0x44, 0x4f, 0x80, 0xe1, 0xc7, 0xc7, 0x54, 0xf7, - 0x1c, 0x7b, 0x8c, 0x48, 0x9c, 0xf7, 0x18, 0x31, 0xf9, 0x12, 0x97, 0x3c, 0xff, 0x25, 0xee, 0x42, - 0xef, 0xd2, 0xe9, 0xf9, 0x0f, 0xff, 0xff, 0xf5, 0x17, 0x93, 0xf2, 0xd7, 0x09, 0x48, 0xb3, 0xe8, - 0xe0, 0x0f, 0x21, 0x65, 0x12, 0xbf, 0xff, 0x4c, 0x8d, 0x00, 0x93, 0x7c, 0x9a, 0x3e, 0xa0, 0x0e, - 0xa9, 0x91, 0xeb, 0xf3, 0x70, 0x3e, 0xf1, 0x5d, 0x6c, 0xcf, 0xf5, 0x83, 0x3d, 0xfe, 0xb6, 0x4e, - 0x33, 0x28, 0xd4, 0x43, 0x85, 0x71, 0x03, 0x32, 0x51, 0x3b, 0x9f, 0x9a, 0xb3, 0x9d, 0x8f, 0x24, - 0xcf, 0xde, 0xe8, 0xd2, 0xcf, 0xf3, 0x46, 0x57, 0xf9, 0xab, 0x04, 0xc5, 0x3d, 0xcf, 0xfd, 0x9c, - 0xf4, 0x83, 0x0e, 0xe1, 0x97, 0xe6, 0x0f, 0x21, 0x4d, 0x33, 0x32, 0xdc, 0x33, 0xf3, 0xa4, 0x24, - 0x17, 0xc4, 0xdb, 0x70, 0x69, 0x40, 0x1c, 0xe2, 0x19, 0x41, 0xac, 0xad, 0xe4, 0x17, 0xcc, 0xf2, - 0x74, 0x60, 0xc8, 0x70, 0x83, 0x7f, 0xbd, 0xe8, 0x6a, 0x28, 0x12, 0x0a, 0xfb, 0xcd, 0x37, 0x01, - 0x39, 0xe3, 0x21, 0x3b, 0x32, 0xf5, 0x11, 0xf1, 0xf4, 0x01, 0x71, 0xf8, 0xe5, 0x52, 0x2b, 0x38, - 0xe3, 0x21, 0x3d, 0x2d, 0xf7, 0x88, 0xb7, 0x4d, 0x9c, 0xca, 0x77, 0x05, 0xc8, 0xdf, 0xb7, 0x1c, - 0xd3, 0x3d, 0x16, 0x3b, 0x7f, 0x8d, 0x3d, 0xba, 0x06, 0x16, 0x3b, 0x97, 0x4f, 0xc5, 0xad, 0x3f, - 0x4e, 0xc2, 0x1d, 0xc8, 0x1e, 0x33, 0x89, 0x66, 0x64, 0xdc, 0xe6, 0xec, 0xa5, 0xc6, 0xc1, 0xc5, - 0xa0, 0x19, 0x1d, 0x37, 0x11, 0x4e, 0xf9, 0xcf, 0x92, 0x38, 0x68, 0x3a, 0x50, 0x08, 0xdb, 0x00, - 0xd2, 0x7c, 0xd6, 0x43, 0x57, 0x9b, 0xc4, 0xc0, 0x77, 0x01, 0x84, 0x2a, 0x8a, 0x98, 0x60, 0x88, - 0xef, 0xcc, 0x67, 0x33, 0x45, 0x8d, 0x81, 0xbc, 0x9f, 0x7a, 0xf4, 0xe5, 0x35, 0xa9, 0xfc, 0xe5, - 0x12, 0xa4, 0x9b, 0x9e, 0x31, 0x24, 0xf8, 0x0e, 0xa4, 0x86, 0xae, 0x49, 0x84, 0xb9, 0x4f, 0x0b, - 0xce, 0x64, 0x37, 0x76, 0x5d, 0x33, 0x2a, 0x2f, 0x14, 0x04, 0xdf, 0x85, 0xc5, 0x7d, 0x77, 0xec, - 0x98, 0xfe, 0x8c, 0x56, 0xef, 0xc9, 0x70, 0x35, 0x26, 0x1a, 0x16, 0x3b, 0x0e, 0x84, 0x3f, 0x81, - 0x2c, 0x39, 0xe9, 0xdb, 0x63, 0x9a, 0x74, 0x6c, 0x1b, 0x16, 0xb7, 0xde, 0x9d, 0x0b, 0x55, 0x0e, - 0xa5, 0xa3, 0x87, 0x8c, 0x90, 0x50, 0xfe, 0x41, 0x82, 0x34, 0x53, 0x4a, 0xb5, 0x30, 0x7d, 0xb4, - 0x60, 0x0a, 0x57, 0xbc, 0x3b, 0xbf, 0xed, 0xb1, 0x72, 0x7b, 0x06, 0x47, 0x0f, 0x06, 0xcb, 0x09, - 0x74, 0xf7, 0xe0, 0xc0, 0x27, 0xbc, 0x7d, 0x0a, 0xbf, 0x7f, 0x64, 0x2d, 0x27, 0x50, 0x19, 0x19, - 0x5f, 0x87, 0x3c, 0xdd, 0x15, 0x66, 0x38, 0x8d, 0xae, 0x34, 0xaf, 0xe5, 0x18, 0x4d, 0x4c, 0xd9, - 0x81, 0x1c, 0x67, 0xb2, 0xcf, 0x85, 0xa2, 0x92, 0xcc, 0xf1, 0x4d, 0x0d, 0xb8, 0x34, 0xb5, 0xa9, - 0xfc, 0x7b, 0x09, 0x16, 0xb9, 0xbb, 0xb1, 0x02, 0x69, 0x3f, 0x30, 0xbc, 0x40, 0x14, 0xd2, 0xad, - 0xf9, 0x97, 0x1d, 0x15, 0x18, 0x0a, 0x83, 0x1b, 0x90, 0x24, 0x8e, 0x29, 0x12, 0xe0, 0x19, 0xd0, - 0x34, 0x2a, 0x5e, 0x79, 0x13, 0x52, 0x34, 0xbb, 0x68, 0xe7, 0xa5, 0x55, 0x95, 0x6d, 0x19, 0x2d, - 0xd0, 0x0e, 0x81, 0x35, 0x49, 0x12, 0xed, 0x10, 0xb6, 0x35, 0xb5, 0xb7, 0xd7, 0x41, 0x89, 0xca, - 0x17, 0x90, 0x8d, 0x7c, 0x8f, 0xaf, 0xc0, 0xe5, 0x9e, 0x52, 0x53, 0x7b, 0x4a, 0x43, 0x6e, 0xe8, - 0x7b, 0x9a, 0x5c, 0x97, 0x1b, 0x2d, 0x65, 0x1b, 0x2d, 0x4c, 0x32, 0x9a, 0x6a, 0xbb, 0xad, 0xde, - 0xa7, 0x0c, 0x89, 0x36, 0x25, 0x6a, 0xb3, 0xd9, 0x91, 0xbb, 0xb1, 0xe9, 0x89, 0x18, 0xf5, 0x6c, - 0x6e, 0x12, 0x2f, 0x43, 0xae, 0xde, 0xd3, 0x34, 0x99, 0x77, 0x6b, 0x28, 0x55, 0xf9, 0x14, 0xb2, - 0x51, 0x76, 0xd1, 0xc6, 0x4c, 0x51, 0x75, 0xf9, 0xa3, 0x7a, 0xbb, 0xd7, 0x69, 0xa9, 0x0a, 0x57, - 0xca, 0x86, 0x0d, 0x59, 0x8f, 0xcb, 0x49, 0xf8, 0x12, 0x14, 0x42, 0x06, 0x5b, 0x07, 0x4a, 0x50, - 0xe9, 0x90, 0xd4, 0x6d, 0xc9, 0x1d, 0x94, 0x2c, 0xff, 0x3d, 0x01, 0x99, 0xb0, 0xee, 0x60, 0x39, - 0xd6, 0xc9, 0xe7, 0xb6, 0xde, 0x7a, 0x5a, 0xaf, 0x4e, 0xf7, 0xf1, 0x17, 0x73, 0x12, 0xd5, 0x20, - 0x7d, 0x40, 0xe3, 0x25, 0xae, 0xa7, 0xb7, 0xe6, 0x89, 0xb1, 0xc6, 0x45, 0xf1, 0x3a, 0x4c, 0xdc, - 0x0c, 0x58, 0x13, 0x92, 0x0e, 0xef, 0x53, 0x13, 0x77, 0x86, 0x32, 0x64, 0x0c, 0x6f, 0xe0, 0xb7, - 0xcc, 0x13, 0xbf, 0xb4, 0xc4, 0xaa, 0x7a, 0x34, 0xa6, 0x28, 0xfc, 0x15, 0x59, 0xa0, 0x64, 0xe2, - 0xb7, 0xf4, 0x38, 0x67, 0x27, 0x95, 0x49, 0xa0, 0xa4, 0xb8, 0x1c, 0xfc, 0x51, 0x02, 0x38, 0xab, - 0x8e, 0xb4, 0xc7, 0xd4, 0xd4, 0xfb, 0xba, 0xd2, 0xdb, 0xad, 0xc9, 0x9a, 0xc8, 0xb3, 0xaa, 0x72, - 0x87, 0x77, 0x9f, 0x0d, 0x59, 0xe9, 0xc8, 0x3a, 0x1b, 0xb3, 0x20, 0x89, 0x86, 0x97, 0x53, 0x92, - 0xb4, 0xbd, 0xae, 0xf7, 0x76, 0x59, 0x5b, 0xdc, 0xe5, 0xf7, 0x04, 0xd6, 0x0c, 0xf3, 0x7b, 0x42, - 0xbb, 0xba, 0x8d, 0x16, 0x29, 0x5c, 0x5b, 0xae, 0x36, 0xd0, 0x12, 0xcd, 0x9f, 0x66, 0x4b, 0xeb, - 0x74, 0xf5, 0x7b, 0xd5, 0x76, 0x4f, 0x46, 0x19, 0x8a, 0xdf, 0xae, 0x46, 0xe3, 0x2c, 0x45, 0x53, - 0xba, 0xb7, 0xc5, 0x10, 0x6e, 0xfe, 0x02, 0x8a, 0x93, 0x1f, 0x33, 0x68, 0xe2, 0xef, 0xf5, 0x6a, - 0xed, 0x56, 0x1d, 0x2d, 0xe0, 0x57, 0xe0, 0x25, 0xfe, 0x9b, 0x76, 0xef, 0xec, 0x82, 0x23, 0x58, - 0x52, 0xed, 0xed, 0x47, 0xdf, 0xad, 0x2e, 0x3c, 0x7a, 0xbc, 0x2a, 0x7d, 0xf3, 0x78, 0x55, 0xfa, - 0xf6, 0xf1, 0xaa, 0xf4, 0xcf, 0xc7, 0xab, 0xd2, 0xaf, 0xbf, 0x5f, 0x5d, 0xf8, 0xe6, 0xfb, 0xd5, - 0x85, 0x6f, 0xbf, 0x5f, 0x5d, 0xf8, 0x24, 0x17, 0xfb, 0x9f, 0x80, 0xff, 0x04, 0x00, 0x00, 0xff, - 0xff, 0x8b, 0x00, 0x06, 0xaf, 0xee, 0x20, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_9f5d15014b584a4d) +} + +var fileDescriptor_processors_sql_9f5d15014b584a4d = []byte{ + // 2682 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0xcd, 0x73, 0xdb, 0xc6, + 0x15, 0x17, 0xf8, 0x21, 0x91, 0x8f, 0x1f, 0x5a, 0xaf, 0x95, 0x98, 0xa1, 0x33, 0xb2, 0xcc, 0xa4, + 0xb1, 0xec, 0x38, 0x52, 0xa3, 0x76, 0xd2, 0x49, 0xd2, 0x43, 0xf8, 0x01, 0xca, 0x94, 0x29, 0x40, + 0x06, 0x49, 0x3b, 0x1f, 0x33, 0xc1, 0x40, 0xc4, 0x8a, 0x42, 0x0c, 0x02, 0x14, 0x00, 0x5a, 0x52, + 0xfe, 0x81, 0x5e, 0xdb, 0xe9, 0xa5, 0xa7, 0x4e, 0x2e, 0x6d, 0xff, 0x81, 0x9e, 0xd3, 0x4b, 0x0f, + 0x3e, 0xe6, 0xd4, 0xe6, 0xd4, 0x69, 0x9c, 0x7f, 0xa0, 0x3d, 0x79, 0x7a, 0xeb, 0xec, 0x07, 0x20, + 0x90, 0x23, 0xba, 0xa6, 0xed, 0xc4, 0xd3, 0x8b, 0x86, 0xfb, 0xbe, 0xf6, 0xed, 0x7b, 0x6f, 0xdf, + 0xfe, 0x76, 0x21, 0x78, 0xd3, 0x3f, 0xb2, 0x37, 0xc9, 0x09, 0xe9, 0x5b, 0xce, 0x81, 0x67, 0x8c, + 0xf6, 0x37, 0x47, 0x9e, 0xdb, 0x27, 0xbe, 0xef, 0x7a, 0xbe, 0xee, 0x1f, 0xd9, 0x1b, 0x23, 0xcf, + 0x0d, 0x5c, 0x5c, 0xea, 0xbb, 0xfd, 0xfb, 0x9e, 0x6b, 0xf4, 0x0f, 0x37, 0x28, 0xd1, 0xb4, 0xfc, + 0xc0, 0x3f, 0xb2, 0xbd, 0xb1, 0x53, 0x7e, 0x9d, 0xea, 0xfb, 0x47, 0xf6, 0xbe, 0xe1, 0x93, 0x4d, + 0x3f, 0xf0, 0xc6, 0xfd, 0x60, 0xec, 0x11, 0x93, 0xeb, 0x95, 0x2f, 0xc7, 0xb9, 0x5f, 0xb8, 0x96, + 0xa3, 0x07, 0xa7, 0x23, 0x22, 0x98, 0xaf, 0xc5, 0x99, 0xb6, 0xdb, 0xbf, 0x6f, 0x39, 0x03, 0xc1, + 0x7a, 0x85, 0xb2, 0xa8, 0xa8, 0xcf, 0xff, 0x0a, 0x72, 0x79, 0xda, 0x59, 0xd3, 0x08, 0x0c, 0xc1, + 0xfb, 0xc9, 0x13, 0x16, 0x42, 0x67, 0x10, 0x62, 0x37, 0xa8, 0x98, 0x3b, 0x0a, 0x36, 0x2d, 0xe7, + 0x01, 0xf1, 0x02, 0x62, 0x92, 0x93, 0x91, 0xb7, 0xe9, 0x8f, 0x0c, 0x47, 0xa7, 0xbf, 0x88, 0xef, + 0x5b, 0xae, 0x23, 0x64, 0x57, 0x06, 0xee, 0xc0, 0x65, 0x3f, 0x37, 0xe9, 0x2f, 0x4e, 0xad, 0xfc, + 0x56, 0x82, 0xe2, 0x5d, 0xc3, 0x1e, 0x13, 0xbf, 0xee, 0x7a, 0xa4, 0x33, 0x22, 0x7d, 0x5c, 0x87, + 0xa5, 0xbe, 0x6b, 0x8f, 0x87, 0x8e, 0x5f, 0x92, 0xd6, 0x92, 0xeb, 0xb9, 0xad, 0x37, 0x36, 0x66, + 0x05, 0x6c, 0xa3, 0x61, 0x04, 0xe3, 0x61, 0xcb, 0x39, 0x70, 0x6b, 0xa9, 0x87, 0xff, 0xb8, 0xb2, + 0xa0, 0x85, 0x9a, 0xf8, 0x32, 0x64, 0x3d, 0xe3, 0x58, 0xdf, 0x3f, 0x0d, 0x88, 0x5f, 0x4a, 0xac, + 0x25, 0xd7, 0xf3, 0x5a, 0xc6, 0x33, 0x8e, 0x6b, 0x74, 0x8c, 0xaf, 0x40, 0xc6, 0x19, 0x0f, 0x75, + 0xcf, 0x3d, 0xf6, 0x4b, 0xc9, 0x35, 0x69, 0x3d, 0x15, 0x6a, 0x3b, 0xe3, 0xa1, 0xe6, 0x1e, 0xfb, + 0x95, 0x7f, 0xa7, 0x61, 0xb9, 0x6b, 0xec, 0xdb, 0x44, 0x23, 0x86, 0x49, 0x3c, 0xe6, 0x56, 0x0d, + 0xd2, 0x01, 0x25, 0x95, 0xa4, 0x35, 0x69, 0x3d, 0xb7, 0xf5, 0xd6, 0x94, 0x53, 0x22, 0xf4, 0x1b, + 0x4c, 0xad, 0x41, 0xfc, 0xbe, 0x67, 0x8d, 0x02, 0xd7, 0x13, 0x96, 0xb9, 0x2a, 0xbe, 0x0a, 0x59, + 0xcb, 0x31, 0xc9, 0x89, 0x6e, 0x99, 0x27, 0xa5, 0xc4, 0x9a, 0xb4, 0x5e, 0x10, 0xfc, 0x0c, 0x23, + 0xb7, 0xcc, 0x13, 0xbc, 0x0a, 0x4b, 0x1e, 0x79, 0x40, 0x3c, 0x9f, 0x30, 0xd7, 0x32, 0xa1, 0x6b, + 0x82, 0x88, 0x65, 0x48, 0xd3, 0xf8, 0xfa, 0xa5, 0x14, 0x8b, 0xcd, 0xf5, 0xd9, 0xb1, 0x99, 0x58, + 0x80, 0xe1, 0x84, 0x9e, 0x30, 0x6d, 0xfc, 0x06, 0x80, 0x6d, 0x0d, 0xad, 0x40, 0x3f, 0xb4, 0x9c, + 0xa0, 0x94, 0x5e, 0x93, 0xd6, 0x93, 0x42, 0x20, 0xcb, 0xe8, 0xb7, 0x2c, 0x27, 0xa0, 0x71, 0xb2, + 0x7c, 0xbd, 0x7f, 0x48, 0xfa, 0xf7, 0x4b, 0x8b, 0x71, 0x67, 0x2c, 0xbf, 0x4e, 0x89, 0x58, 0x01, + 0x78, 0x60, 0xf9, 0xd6, 0xbe, 0x65, 0x5b, 0xc1, 0x69, 0x69, 0x69, 0x4d, 0x5a, 0x2f, 0x6e, 0xad, + 0xcf, 0xf6, 0xa8, 0xd3, 0x37, 0x9c, 0xbb, 0x91, 0xbc, 0x30, 0x16, 0xb3, 0x80, 0x3f, 0x84, 0x4b, + 0x43, 0xe3, 0x44, 0x0f, 0xac, 0x21, 0xf1, 0x03, 0x63, 0x38, 0xd2, 0x8d, 0x01, 0xd1, 0x1d, 0xc3, + 0x71, 0xfd, 0x52, 0x36, 0x96, 0xa7, 0x95, 0xa1, 0x71, 0xd2, 0x0d, 0x65, 0xaa, 0x03, 0xa2, 0x50, + 0x09, 0xfc, 0x19, 0x20, 0x51, 0xf7, 0xba, 0x1f, 0x78, 0xc4, 0x19, 0x04, 0x87, 0x25, 0x60, 0x2e, + 0xdd, 0x98, 0x91, 0x2b, 0xea, 0x4f, 0x9b, 0xab, 0x74, 0x84, 0x86, 0x98, 0x61, 0xd9, 0x9e, 0x24, + 0xe3, 0x7d, 0xb8, 0x18, 0x1a, 0x3f, 0x36, 0xac, 0x40, 0x1f, 0xb9, 0xb6, 0xd5, 0x3f, 0x2d, 0xe5, + 0x98, 0xfd, 0x9b, 0xff, 0xdb, 0xfe, 0x3d, 0xc3, 0x0a, 0xf6, 0x98, 0x8e, 0x98, 0xe1, 0x82, 0x3d, + 0xcd, 0xc0, 0x6f, 0x41, 0x6e, 0x64, 0x78, 0x86, 0x6d, 0x13, 0xdb, 0xfa, 0x92, 0x94, 0xf2, 0xb1, + 0x88, 0xc7, 0x19, 0x58, 0x81, 0xa2, 0x7f, 0xea, 0x07, 0x64, 0xa8, 0x87, 0xfb, 0xa4, 0xb0, 0x96, + 0x5c, 0x2f, 0x6e, 0x5d, 0x9b, 0xe5, 0x06, 0x13, 0xae, 0x33, 0xd9, 0xdb, 0x96, 0x63, 0x6a, 0x05, + 0x3f, 0x46, 0xf1, 0x77, 0x52, 0x99, 0x0c, 0xca, 0x56, 0xfe, 0x93, 0x84, 0x52, 0x8b, 0x56, 0x61, + 0xe7, 0xbe, 0x35, 0x7a, 0x49, 0xc5, 0x1f, 0x15, 0x77, 0xf2, 0xb9, 0x8a, 0x7b, 0xb2, 0x2c, 0x53, + 0xcf, 0x5d, 0x96, 0xb1, 0x3d, 0x99, 0x3e, 0x6f, 0x4f, 0x9e, 0x57, 0x79, 0x8b, 0x3f, 0x70, 0xe5, + 0x2d, 0xbd, 0xc0, 0xca, 0xab, 0x3c, 0x4e, 0x43, 0x71, 0xc7, 0xb5, 0x9c, 0x1f, 0x3f, 0xe3, 0xd7, + 0xa1, 0x68, 0xbb, 0xee, 0xfd, 0xf1, 0x28, 0xaa, 0x65, 0x9a, 0xfa, 0x42, 0x2d, 0x81, 0x24, 0xad, + 0xc0, 0x39, 0xa2, 0x4c, 0xe9, 0xb9, 0xe0, 0xf2, 0x73, 0x85, 0xa5, 0x34, 0xb7, 0xf5, 0xe6, 0xec, + 0x94, 0xca, 0xd1, 0xe9, 0x23, 0x66, 0x5c, 0x74, 0x1d, 0x4a, 0xc3, 0xef, 0x43, 0x8a, 0x9e, 0x81, + 0x22, 0x3d, 0x57, 0x66, 0xac, 0x8a, 0xc6, 0xa2, 0x7b, 0x3a, 0x22, 0x42, 0x99, 0xa9, 0xbc, 0xf0, + 0x66, 0xf7, 0x3e, 0xbc, 0x3a, 0xb9, 0x74, 0xdd, 0xf0, 0x88, 0x7e, 0x9f, 0x9c, 0x96, 0x32, 0xb1, + 0x22, 0xbb, 0x38, 0x11, 0x84, 0xaa, 0x47, 0x6e, 0x93, 0xd3, 0x73, 0x0b, 0x2e, 0xfb, 0x03, 0x17, + 0x1c, 0xbc, 0xc8, 0x56, 0xf7, 0x2e, 0x5c, 0x18, 0x1a, 0x96, 0x13, 0x18, 0x96, 0xa3, 0xbb, 0x9e, + 0x49, 0x3c, 0xcb, 0x19, 0xb0, 0x66, 0x1a, 0x2e, 0x1b, 0x85, 0x6c, 0x55, 0x70, 0xcf, 0xe9, 0x7a, + 0xf9, 0xe7, 0xec, 0x7a, 0x69, 0xb4, 0x48, 0xf1, 0x07, 0x74, 0x5c, 0x2f, 0x10, 0x55, 0x7f, 0x07, + 0x96, 0xdd, 0x71, 0x30, 0x1a, 0x07, 0x67, 0x5e, 0xf1, 0xfa, 0xaf, 0xcc, 0x4e, 0x74, 0xe8, 0xa1, + 0xf0, 0xbc, 0xc8, 0x0d, 0x44, 0x7e, 0x6f, 0x01, 0x0e, 0x6d, 0xe9, 0x43, 0x23, 0xe8, 0x1f, 0xea, + 0x36, 0x71, 0x26, 0x76, 0x03, 0x0a, 0xf9, 0xbb, 0x94, 0xdd, 0x26, 0x4e, 0xe5, 0x6b, 0x09, 0xf2, + 0x0d, 0xcb, 0x0f, 0x2c, 0xa7, 0x1f, 0x30, 0xbf, 0xae, 0xc1, 0x32, 0x13, 0x22, 0xa6, 0x1e, 0xc7, + 0x46, 0x05, 0xad, 0x28, 0xc8, 0xe1, 0x26, 0xb9, 0x0e, 0xc8, 0x14, 0x8a, 0x91, 0x64, 0x82, 0x49, + 0x2e, 0x87, 0xf4, 0x50, 0x74, 0x0b, 0xb0, 0x33, 0xb6, 0x6d, 0x5e, 0x76, 0x21, 0x73, 0x02, 0x74, + 0x20, 0xc6, 0xaf, 0x7a, 0x24, 0xf4, 0x05, 0xbf, 0x05, 0x79, 0xe2, 0x79, 0xae, 0xa7, 0xbb, 0x8e, + 0x6e, 0x8e, 0x47, 0x6c, 0x23, 0x66, 0xc3, 0xda, 0x66, 0x1c, 0xd5, 0x69, 0x8c, 0x47, 0x15, 0x04, + 0x45, 0xd5, 0x33, 0x2d, 0xc7, 0xa0, 0x95, 0x4e, 0x57, 0x50, 0xf9, 0x5d, 0x12, 0xd0, 0xa7, 0xd6, + 0xe0, 0x4b, 0x63, 0x40, 0x37, 0x97, 0x08, 0x77, 0x03, 0x16, 0x59, 0xa7, 0x08, 0x91, 0xde, 0x7c, + 0x5d, 0x46, 0xe8, 0xe2, 0x26, 0x00, 0x39, 0x9a, 0x58, 0x6d, 0x6e, 0xeb, 0xea, 0xec, 0x7c, 0x89, + 0xf5, 0x87, 0x70, 0x87, 0x1c, 0x9d, 0xc5, 0xae, 0xc8, 0xdb, 0x95, 0xcb, 0x5d, 0x9f, 0xe8, 0x45, + 0x8c, 0x23, 0xd6, 0xf4, 0x82, 0x7a, 0xd1, 0x6d, 0xc8, 0x1f, 0x58, 0x27, 0xc4, 0xd4, 0x1f, 0x30, + 0x00, 0x5c, 0x4a, 0x33, 0xcf, 0x9f, 0xd0, 0x52, 0x26, 0x81, 0xb2, 0x96, 0x63, 0xda, 0x9c, 0xf8, + 0x1c, 0x8d, 0xad, 0xf2, 0xb7, 0x24, 0x2c, 0xef, 0x12, 0x6f, 0x40, 0x62, 0x99, 0xd9, 0x85, 0x82, + 0x4d, 0x0e, 0x9e, 0x63, 0x1b, 0xe4, 0xa9, 0x7a, 0xb4, 0x09, 0x54, 0x28, 0x7a, 0xd6, 0xe0, 0x30, + 0x66, 0x2f, 0x31, 0xa7, 0xbd, 0x02, 0xd3, 0x8f, 0x0c, 0xc6, 0x12, 0x90, 0x7e, 0x19, 0x87, 0xc1, + 0x75, 0x28, 0xd0, 0xcd, 0xa1, 0x93, 0xa3, 0xb1, 0x11, 0x9d, 0x07, 0xe1, 0xbe, 0xc9, 0x53, 0x96, + 0x2c, 0x38, 0xf8, 0x03, 0xb8, 0xc4, 0x42, 0x79, 0x56, 0xa3, 0x33, 0x1a, 0x3d, 0x39, 0x08, 0xe4, + 0xa3, 0xc9, 0x46, 0xff, 0x4b, 0x28, 0xf1, 0xb8, 0x9d, 0xa3, 0x9c, 0x8d, 0x29, 0xaf, 0x30, 0xa9, + 0x29, 0xed, 0xca, 0x6f, 0x92, 0x50, 0xbc, 0x65, 0xf8, 0x87, 0xb1, 0xbc, 0xde, 0x80, 0xe5, 0x29, + 0x67, 0x78, 0x23, 0x11, 0x07, 0x6e, 0xdc, 0x05, 0x7c, 0x13, 0xd0, 0xf4, 0xe4, 0xbc, 0x97, 0x30, + 0xe1, 0xe2, 0xe4, 0x94, 0x2f, 0x3d, 0x23, 0x6f, 0x43, 0x71, 0x48, 0x8b, 0xf8, 0xac, 0x43, 0xc6, + 0x53, 0x52, 0xe0, 0xbc, 0xd0, 0xd9, 0x97, 0x97, 0x93, 0xc7, 0x09, 0xc0, 0x2d, 0x71, 0x5b, 0x8e, + 0xe5, 0xe5, 0x47, 0x83, 0x5b, 0x85, 0x09, 0xcc, 0xc1, 0xda, 0x7d, 0x21, 0xda, 0xb2, 0x31, 0xa8, + 0x81, 0x55, 0x28, 0x84, 0xb7, 0xfa, 0x67, 0x6d, 0x74, 0xf9, 0xd0, 0x00, 0xcb, 0xed, 0x4b, 0x2e, + 0x90, 0xca, 0x1f, 0x25, 0x58, 0x09, 0x23, 0xdf, 0xb4, 0xec, 0x80, 0x78, 0x22, 0xf6, 0x1b, 0x80, + 0xa2, 0x95, 0xf6, 0x5d, 0x9b, 0x85, 0x4f, 0x8a, 0xc5, 0xa5, 0x18, 0x72, 0xeb, 0xae, 0x4d, 0x83, + 0xf8, 0xf9, 0x74, 0x64, 0x78, 0x2f, 0xfb, 0xd9, 0x94, 0x33, 0xee, 0x28, 0xd8, 0x88, 0xbf, 0x8b, + 0x6c, 0xd0, 0x8b, 0xca, 0xd9, 0xc2, 0xf6, 0x3c, 0x37, 0x70, 0xcf, 0x0b, 0x54, 0xe5, 0x5f, 0x4b, + 0x50, 0xac, 0x0e, 0x06, 0x1e, 0x19, 0x18, 0x81, 0xcb, 0x5d, 0xbc, 0x0a, 0x30, 0xf0, 0x5c, 0x9e, + 0xb6, 0xf8, 0x26, 0xcc, 0x32, 0x6a, 0xdd, 0xb5, 0x7d, 0xfc, 0x39, 0xe4, 0x0d, 0xa1, 0x64, 0xb9, + 0xd1, 0x15, 0xea, 0xe7, 0xb3, 0x63, 0x3c, 0x39, 0x45, 0x34, 0x8c, 0xa5, 0x2f, 0x6e, 0x0f, 0xff, + 0x54, 0xe0, 0x18, 0x62, 0xea, 0x31, 0x57, 0x52, 0x91, 0x2b, 0x48, 0x70, 0xb7, 0x23, 0x8f, 0xb6, + 0x45, 0xae, 0xd2, 0x2c, 0x57, 0xef, 0x3c, 0xb5, 0x27, 0xd3, 0x99, 0x2b, 0xff, 0x2a, 0x01, 0xb9, + 0x98, 0x7b, 0xd4, 0xf0, 0xc1, 0xd8, 0xe9, 0xb3, 0x24, 0xcd, 0x63, 0xb8, 0x39, 0x76, 0xfa, 0xa1, + 0x61, 0x6a, 0x00, 0xaf, 0x41, 0x26, 0x02, 0x3e, 0x89, 0xd8, 0xd6, 0x8d, 0xa8, 0xf8, 0x4d, 0x28, + 0x1e, 0xb0, 0x5a, 0x89, 0x2a, 0x83, 0x6e, 0x83, 0x82, 0x96, 0xe7, 0x54, 0x51, 0x11, 0x97, 0xd8, + 0x93, 0x15, 0x63, 0xa7, 0x19, 0xd8, 0x5a, 0xec, 0x73, 0xc6, 0x2d, 0xc8, 0x1a, 0xde, 0x60, 0x3c, + 0x24, 0x4e, 0xe0, 0x97, 0x16, 0x59, 0x46, 0xe6, 0xa9, 0xfa, 0x33, 0xe5, 0x9d, 0x54, 0x26, 0x89, + 0x52, 0x95, 0x3f, 0x25, 0x21, 0x45, 0x57, 0x81, 0x11, 0xe4, 0xab, 0xca, 0x27, 0xba, 0xa2, 0x76, + 0x75, 0xa5, 0xd7, 0x6e, 0xa3, 0x05, 0xbc, 0x04, 0xc9, 0xea, 0xdd, 0x6d, 0x24, 0xe1, 0x3c, 0x64, + 0x6a, 0xaa, 0xda, 0xd6, 0xab, 0x4a, 0x03, 0x25, 0x70, 0x0e, 0x96, 0xd8, 0x48, 0xd5, 0x50, 0x12, + 0x17, 0x01, 0xea, 0xaa, 0x52, 0xaf, 0x76, 0xf5, 0xea, 0xf6, 0x36, 0x4a, 0xe1, 0x2c, 0xa4, 0xeb, + 0x6a, 0x4f, 0xe9, 0xa2, 0x34, 0x55, 0xdf, 0xad, 0x7e, 0x8c, 0x96, 0xd8, 0x8f, 0x96, 0x82, 0x32, + 0x18, 0x60, 0xb1, 0xd3, 0x6d, 0x34, 0xe4, 0xbb, 0x28, 0x4b, 0x89, 0x9d, 0xde, 0x2e, 0x02, 0x6a, + 0xae, 0xd3, 0xdb, 0xd5, 0x5b, 0x4a, 0x17, 0xe5, 0xe8, 0x4c, 0x77, 0xab, 0x5a, 0xab, 0xaa, 0xd4, + 0x65, 0x94, 0xa7, 0xac, 0x8f, 0x55, 0x8d, 0x59, 0x2e, 0xf0, 0x99, 0x7a, 0x4a, 0x57, 0xd7, 0xd4, + 0x7b, 0x1d, 0x54, 0x64, 0x7a, 0x77, 0xb4, 0x46, 0xab, 0xd9, 0x44, 0xcb, 0x18, 0x43, 0xb1, 0xd9, + 0x52, 0xaa, 0x6d, 0x3d, 0xd2, 0x46, 0x74, 0x41, 0x9c, 0x26, 0xe6, 0xbc, 0x80, 0x0b, 0x90, 0xad, + 0x6a, 0x5a, 0xf5, 0x13, 0x66, 0x11, 0xd3, 0xc9, 0x76, 0x3a, 0xaa, 0xc2, 0x46, 0x17, 0x29, 0x93, + 0x8e, 0x6a, 0x6c, 0xb8, 0x42, 0xa7, 0xeb, 0x74, 0xb5, 0x96, 0xb2, 0xcd, 0xc6, 0xaf, 0xb0, 0x55, + 0xb7, 0xba, 0x2c, 0x04, 0xaf, 0xd2, 0x85, 0xd0, 0x81, 0xaa, 0xa1, 0x4b, 0x38, 0x03, 0xa9, 0xba, + 0xaa, 0x69, 0xa8, 0x84, 0x4b, 0xb0, 0xb2, 0x27, 0x6b, 0x75, 0x59, 0xe9, 0xb6, 0xda, 0xb2, 0xde, + 0x68, 0x75, 0xea, 0x7a, 0x6b, 0x77, 0xaf, 0x8d, 0x5e, 0x9b, 0xe2, 0xd4, 0x55, 0xa5, 0xcb, 0x39, + 0x65, 0x7c, 0x11, 0x96, 0x99, 0x0f, 0x6a, 0x6d, 0x47, 0xae, 0xf3, 0x20, 0x5e, 0xc6, 0x2b, 0x80, + 0xb8, 0x2b, 0x31, 0xea, 0xeb, 0x95, 0x9b, 0x90, 0xa2, 0x75, 0x4c, 0x27, 0xac, 0xf6, 0xba, 0x2a, + 0x5a, 0x60, 0xf1, 0xac, 0x57, 0xdb, 0x55, 0x0d, 0x49, 0xd4, 0x5f, 0x45, 0x55, 0x74, 0x31, 0x4e, + 0x54, 0xfe, 0xb2, 0x08, 0x97, 0x5b, 0x4e, 0x40, 0x3c, 0x9b, 0x18, 0x0f, 0x88, 0xc9, 0xef, 0xe1, + 0xb1, 0xe3, 0xe1, 0x93, 0x29, 0xa0, 0xfc, 0xe1, 0xec, 0x22, 0x7a, 0x82, 0x19, 0x7e, 0x7c, 0x4c, + 0xa1, 0xe7, 0xd8, 0xe3, 0x46, 0xe2, 0xbc, 0xc7, 0x8d, 0xc9, 0x97, 0xc2, 0xe4, 0xf9, 0x2f, 0x85, + 0x2f, 0xf4, 0x6e, 0x9e, 0x9e, 0xff, 0xf0, 0xff, 0x7f, 0x7f, 0x81, 0x29, 0x7f, 0x9d, 0x80, 0x34, + 0xcb, 0x0e, 0xfe, 0x08, 0x52, 0x26, 0xf1, 0xfb, 0xcf, 0x04, 0x04, 0x98, 0xe6, 0xd3, 0xe0, 0x80, + 0x3a, 0xa4, 0x46, 0xae, 0xcf, 0xd3, 0xf9, 0xc4, 0x77, 0xb6, 0x3d, 0xd7, 0x0f, 0xf6, 0xf8, 0xdb, + 0x3f, 0xad, 0xa0, 0x70, 0x1e, 0xaa, 0x8c, 0x1b, 0x90, 0x89, 0xe0, 0x7c, 0x6a, 0x4e, 0x38, 0x1f, + 0x69, 0x9e, 0xbd, 0xf9, 0xa5, 0x9f, 0xe7, 0xcd, 0xaf, 0xf2, 0x57, 0x09, 0x8a, 0x7b, 0x9e, 0xfb, + 0x05, 0xe9, 0x07, 0x1d, 0xc2, 0x2f, 0xcd, 0x1f, 0x41, 0x9a, 0x56, 0x64, 0xb8, 0x67, 0xe6, 0x29, + 0x49, 0xae, 0x88, 0xb7, 0xe1, 0xc2, 0x80, 0x38, 0xc4, 0x33, 0x82, 0x18, 0xac, 0xe4, 0x17, 0xcc, + 0xf2, 0x74, 0x62, 0xc8, 0x70, 0x83, 0x7f, 0x5d, 0xe9, 0x6a, 0x28, 0x52, 0x0a, 0xf1, 0xe6, 0x35, + 0x40, 0xce, 0x98, 0xbd, 0x5c, 0xf8, 0xfa, 0x88, 0x78, 0xfa, 0x80, 0x38, 0xfc, 0x72, 0xa9, 0x15, + 0x9c, 0xf1, 0x90, 0x9e, 0x96, 0x7b, 0xc4, 0xdb, 0x26, 0x4e, 0xe5, 0xbb, 0x02, 0xe4, 0xef, 0x59, + 0x8e, 0xe9, 0x1e, 0x8b, 0x9d, 0xbf, 0xc6, 0x1e, 0x85, 0x03, 0x8b, 0x9d, 0xcb, 0xa7, 0xe2, 0xd6, + 0x1f, 0x27, 0xe1, 0x0e, 0x64, 0x8f, 0x99, 0x46, 0x33, 0x72, 0x6e, 0x73, 0xf6, 0x52, 0xe3, 0xc6, + 0xc5, 0xa0, 0x19, 0x1d, 0x37, 0x91, 0x9d, 0xf2, 0x9f, 0x25, 0x71, 0xd0, 0x74, 0xa0, 0x10, 0xc2, + 0x00, 0xd2, 0x7c, 0xd6, 0x43, 0x57, 0x9b, 0xb4, 0x81, 0xef, 0x00, 0x88, 0xa9, 0xa8, 0xc5, 0x04, + 0xb3, 0xf8, 0xee, 0x7c, 0x3e, 0x53, 0xab, 0x31, 0x23, 0x1f, 0xa4, 0x1e, 0x7e, 0x75, 0x45, 0x2a, + 0x7f, 0xb5, 0x04, 0xe9, 0xa6, 0x67, 0x0c, 0x09, 0xbe, 0x0d, 0xa9, 0xa1, 0x6b, 0x12, 0xe1, 0xee, + 0xd3, 0x1a, 0x67, 0xba, 0x1b, 0xbb, 0xae, 0x19, 0xb5, 0x17, 0x6a, 0x04, 0xdf, 0x81, 0xc5, 0x7d, + 0x77, 0xec, 0x98, 0xfe, 0x0c, 0xa8, 0xf7, 0x64, 0x73, 0x35, 0xa6, 0x1a, 0x36, 0x3b, 0x6e, 0x08, + 0x7f, 0x0a, 0x59, 0x72, 0xd2, 0xb7, 0xc7, 0xb4, 0xe8, 0xd8, 0x36, 0x2c, 0x6e, 0xbd, 0x37, 0x97, + 0x55, 0x39, 0xd4, 0x8e, 0x1e, 0x32, 0x42, 0x42, 0xf9, 0xb1, 0x04, 0x69, 0x36, 0x29, 0x9d, 0x85, + 0xcd, 0x47, 0x1b, 0xa6, 0x08, 0xc5, 0x7b, 0xf3, 0xfb, 0x1e, 0x6b, 0xb7, 0x67, 0xe6, 0xe8, 0xc1, + 0x60, 0x39, 0x81, 0xee, 0x1e, 0x1c, 0xf8, 0x84, 0xc3, 0xa7, 0xf0, 0xfb, 0x4c, 0xd6, 0x72, 0x02, + 0x95, 0x91, 0xf1, 0x55, 0xc8, 0xd3, 0x5d, 0x61, 0x86, 0x62, 0x74, 0xa5, 0x79, 0x2d, 0xc7, 0x68, + 0x42, 0x64, 0x07, 0x72, 0x9c, 0xc9, 0x3e, 0x67, 0x8a, 0x4e, 0x32, 0xc7, 0x37, 0x3f, 0xe0, 0xda, + 0xd4, 0xa7, 0xf2, 0xef, 0x25, 0x58, 0xe4, 0xe1, 0xc6, 0x0a, 0xa4, 0xfd, 0xc0, 0xf0, 0x02, 0xd1, + 0x48, 0xb7, 0xe6, 0x5f, 0x76, 0xd4, 0x60, 0xa8, 0x19, 0xdc, 0x80, 0x24, 0x71, 0x4c, 0x51, 0x00, + 0xcf, 0x60, 0x4d, 0xa3, 0xea, 0x95, 0x6b, 0x90, 0xa2, 0xd5, 0x45, 0x91, 0x97, 0x56, 0x55, 0xb6, + 0x65, 0xb4, 0x40, 0x11, 0x02, 0x03, 0x49, 0x12, 0x45, 0x08, 0xdb, 0x9a, 0xda, 0xdb, 0xeb, 0xa0, + 0x44, 0xe5, 0x4b, 0xc8, 0x46, 0xb1, 0xc7, 0x97, 0xe0, 0x62, 0x4f, 0xa9, 0xa9, 0x3d, 0xa5, 0x21, + 0x37, 0xf4, 0x3d, 0x4d, 0xae, 0xcb, 0x8d, 0x96, 0xb2, 0x8d, 0x16, 0x26, 0x19, 0x4d, 0xb5, 0xdd, + 0x56, 0xef, 0x51, 0x86, 0x44, 0x41, 0x89, 0xda, 0x6c, 0x76, 0xe4, 0x6e, 0x4c, 0x3c, 0x11, 0xa3, + 0x9e, 0xc9, 0x26, 0xf1, 0x32, 0xe4, 0xea, 0x3d, 0x4d, 0x93, 0x39, 0x5a, 0x43, 0xa9, 0xca, 0x67, + 0x90, 0x8d, 0xaa, 0x8b, 0x02, 0x33, 0x45, 0xd5, 0xe5, 0x8f, 0xeb, 0xed, 0x5e, 0xa7, 0xa5, 0x2a, + 0x7c, 0x52, 0x36, 0x6c, 0xc8, 0x7a, 0x5c, 0x4f, 0xc2, 0x17, 0xa0, 0x10, 0x32, 0xd8, 0x3a, 0x50, + 0x82, 0x6a, 0x87, 0xa4, 0x6e, 0x4b, 0xee, 0xa0, 0x64, 0xf9, 0xef, 0x09, 0xc8, 0x84, 0x7d, 0x07, + 0xcb, 0x31, 0x24, 0x9f, 0xdb, 0x7a, 0xfb, 0x69, 0xa3, 0x3a, 0x8d, 0xe3, 0x5f, 0xcc, 0x49, 0x54, + 0x83, 0xf4, 0x01, 0xcd, 0x97, 0xb8, 0x9e, 0xde, 0x9c, 0x27, 0xc7, 0x1a, 0x57, 0xc5, 0xeb, 0x30, + 0x71, 0x33, 0x60, 0x20, 0x24, 0x1d, 0xde, 0xa7, 0x26, 0xee, 0x0c, 0x65, 0xc8, 0x18, 0xde, 0xc0, + 0x6f, 0x99, 0x27, 0x7e, 0x69, 0x89, 0x75, 0xf5, 0x68, 0x4c, 0xad, 0xf0, 0x57, 0x64, 0x61, 0x25, + 0x13, 0xbf, 0xa5, 0xc7, 0x39, 0x3b, 0xa9, 0x4c, 0x02, 0x25, 0xc5, 0xe5, 0xe0, 0x0f, 0x12, 0xc0, + 0x59, 0x77, 0xa4, 0x18, 0x53, 0x53, 0xef, 0xe9, 0x4a, 0x6f, 0xb7, 0x26, 0x6b, 0xa2, 0xce, 0xaa, + 0xca, 0x6d, 0x8e, 0x3e, 0x1b, 0xb2, 0xd2, 0x91, 0x75, 0x36, 0x66, 0x49, 0x12, 0x80, 0x97, 0x53, + 0x92, 0x14, 0x5e, 0xd7, 0x7b, 0xbb, 0x0c, 0x16, 0x77, 0xf9, 0x3d, 0x81, 0x81, 0x61, 0x7e, 0x4f, + 0x68, 0x57, 0xb7, 0xd1, 0x22, 0x35, 0xd7, 0x96, 0xab, 0x0d, 0xb4, 0x44, 0xeb, 0xa7, 0xd9, 0xd2, + 0x3a, 0x5d, 0xfd, 0x6e, 0xb5, 0xdd, 0x93, 0x51, 0x86, 0xda, 0x6f, 0x57, 0xa3, 0x71, 0x96, 0x5a, + 0x53, 0xba, 0xb7, 0xc4, 0x10, 0x6e, 0xfc, 0x02, 0x8a, 0x93, 0x1f, 0x47, 0x68, 0xe1, 0xef, 0xf5, + 0x6a, 0xed, 0x56, 0x1d, 0x2d, 0xe0, 0xd7, 0xe0, 0x15, 0xfe, 0x9b, 0xa2, 0x77, 0x76, 0xc1, 0x11, + 0x2c, 0xa9, 0xf6, 0xce, 0xc3, 0xef, 0x56, 0x17, 0x1e, 0x3e, 0x5a, 0x95, 0xbe, 0x79, 0xb4, 0x2a, + 0x7d, 0xfb, 0x68, 0x55, 0xfa, 0xe7, 0xa3, 0x55, 0xe9, 0xd7, 0xdf, 0xaf, 0x2e, 0x7c, 0xf3, 0xfd, + 0xea, 0xc2, 0xb7, 0xdf, 0xaf, 0x2e, 0x7c, 0x9a, 0x8b, 0xfd, 0xcf, 0xc2, 0x7f, 0x03, 0x00, 0x00, + 0xff, 0xff, 0x0b, 0x06, 0x0e, 0xf5, 0x8e, 0x21, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index b71e181bdfb3..0c82711521d0 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -137,6 +137,11 @@ message TableReaderSpec { // makes it out of the SQL optimizer without throwing an error. If/when other // wait policies are supported, this field will be plumbed further. optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 11 [(gogoproto.nullable) = false]; + + // Indicates what implicit system columns this TableReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the TableReader, in the order specified. + repeated sqlbase.SystemColumnKind system_columns = 13; } // IndexSkipTableReaderSpec is the specification for a table reader that @@ -257,6 +262,15 @@ message JoinReaderSpec { // stream. This is only applicable to lookup joins, where doing so is // expensive. Index joins do this by default regardless of the parameter. optional bool maintain_ordering = 11 [(gogoproto.nullable) = false]; + + // Indicates what implicit system columns this JoinReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the JoinReader, in the order specified. + // + // This is only used in the special case of index joins, where the final + // result of the secondary index joined against the primary index is + // expected to contain the materialized system columns. + repeated sqlbase.SystemColumnKind system_columns = 12; } // SorterSpec is the specification for a "sorting aggregator". A sorting diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index 737004efec59..e8153bbf2218 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -99,7 +99,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv │ ├ *colexec.countOp │ │ └ *colexec.simpleProjectOp │ │ └ *colexec.CancelChecker -│ │ └ *colfetcher.colBatchScan +│ │ └ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox @@ -110,28 +110,28 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox │ └ *colexec.deselectorOp │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox │ └ *colexec.deselectorOp │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox └ *colexec.deselectorOp └ *colexec.countOp └ *colexec.simpleProjectOp └ *colexec.CancelChecker - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 @@ -151,7 +151,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ │ ├ *colexec.routerOutputOp │ │ │ │ │ └ *colexec.HashRouter │ │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ │ └ *colfetcher.ColBatchScan │ │ │ │ ├ *colrpc.Inbox │ │ │ │ ├ *colrpc.Inbox │ │ │ │ ├ *colrpc.Inbox @@ -160,7 +160,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox @@ -186,7 +186,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox │ │ │ └ *colrpc.Inbox @@ -195,7 +195,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ ├ *colrpc.Inbox │ │ ├ *colrpc.Inbox │ │ └ *colrpc.Inbox @@ -217,7 +217,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ └ *colrpc.Inbox │ │ └ *colexec.ParallelUnorderedSynchronizer @@ -226,7 +226,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ ├ *colrpc.Inbox │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer @@ -248,7 +248,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ └ *colrpc.Inbox │ │ └ *colexec.ParallelUnorderedSynchronizer │ │ ├ *colrpc.Inbox @@ -257,7 +257,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer │ ├ *colexec.ParallelUnorderedSynchronizer @@ -279,7 +279,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ └ *colexec.routerOutputOp │ │ └ *colexec.HashRouter │ │ └ *colexec.CancelChecker - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.ParallelUnorderedSynchronizer │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox @@ -288,7 +288,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ └ *colexec.routerOutputOp │ └ *colexec.HashRouter │ └ *colexec.CancelChecker - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.ParallelUnorderedSynchronizer ├ *colexec.ParallelUnorderedSynchronizer └ *colexec.externalHashJoiner diff --git a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node index 8cfd2931c0e2..bb7f7ed02b52 100644 --- a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node +++ b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node @@ -75,23 +75,23 @@ EXPLAIN (VEC) SELECT * FROM kv WHERE k::REGCLASS IS NOT NULL │ └ *colexec.isNullSelOp │ └ *colexec.castOp │ └ *colexec.ParallelUnorderedSynchronizer -│ ├ *colfetcher.colBatchScan +│ ├ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ └ *colrpc.Inbox ├ Node 2 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Check that the plan is local when experimental DistSQL planning is disabled. statement ok @@ -104,7 +104,7 @@ EXPLAIN (VEC) SELECT * FROM kv WHERE k::REGCLASS IS NOT NULL └ Node 1 └ *colexec.isNullSelOp └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET experimental_distsql_planning = always @@ -118,23 +118,23 @@ EXPLAIN (VEC) SELECT k::REGCLASS FROM kv ├ Node 1 │ └ *colexec.castOp │ └ *colexec.ParallelUnorderedSynchronizer -│ ├ *colfetcher.colBatchScan +│ ├ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ └ *colrpc.Inbox ├ Node 2 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query TTT EXPLAIN SELECT k::REGCLASS FROM kv @@ -173,7 +173,14 @@ EXPLAIN (VEC) SELECT k::REGCLASS FROM kv │ └ Node 1 └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET disable_partially_distributed_plans = false + +# Ensure that the new factory can materialize system columns. +# TODO (rohany): The factory can't plan lookup joins yet. Add a test when it can. +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM kv LIMIT 1 +---- +true diff --git a/pkg/sql/logictest/testdata/logic_test/mvcc b/pkg/sql/logictest/testdata/logic_test/mvcc new file mode 100644 index 000000000000..ac250e863c6b --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/mvcc @@ -0,0 +1,124 @@ +statement ok +CREATE TABLE t (x INT PRIMARY KEY, y INT, z INT, INDEX i (z)); +INSERT INTO t VALUES (1, 2, 3) + +# Get the timestamp for row (1, 2, 3). +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM t +---- +true + +let $base_ts +SELECT crdb_internal_mvcc_timestamp FROM t + +# Insert a new value into t. +statement ok +INSERT INTO t VALUES (2, 3, 4) + +# Its timestamp should be bigger than the timestamp of the first row. +query B +SELECT crdb_internal_mvcc_timestamp > $base_ts FROM t WHERE x = 2 +---- +true + +# Check that trying to get the timestamp from t@i succeeds too. +query B +SELECT crdb_internal_mvcc_timestamp = $base_ts FROM t@i WHERE x = 1 +---- +true + +# We should be able to produce the mvcc column as many times as needed. +query BBBB +SELECT crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts FROM t@i WHERE x = 1 +---- +true true true true + +# Update the original row. +statement ok +UPDATE t SET z = 5 WHERE x = 1 + +query B +SELECT crdb_internal_mvcc_timestamp > $base_ts FROM t +---- +true +true + +# Ensure that we can use the timestamp column in different places. +query III rowsort +SELECT * FROM t WHERE crdb_internal_mvcc_timestamp IS NOT NULL +---- +1 2 5 +2 3 4 + +query IIIIII rowsort +SELECT t1.*, t2.* FROM t t1 JOIN t t2 ON t1.crdb_internal_mvcc_timestamp = t2.crdb_internal_mvcc_timestamp +---- +1 2 5 1 2 5 +2 3 4 2 3 4 + +let $update_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 2 + +statement ok +UPDATE t SET z = 6 WHERE crdb_internal_mvcc_timestamp = $update_ts + +query III rowsort +SELECT * FROM t +---- +1 2 5 +2 3 6 + +let $del_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 2 + +statement ok +DELETE FROM t WHERE crdb_internal_mvcc_timestamp = $del_ts + +query III +SELECT * FROM t +---- +1 2 5 + +let $update_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 1 + +statement ok +UPDATE t SET z = (crdb_internal_mvcc_timestamp + 1.0)::INT WHERE x = 1 + +query B +SELECT z = ($update_ts + 1.0)::INT FROM t WHERE x = 1 +---- +true + +query IIB +SELECT x, y, crdb_internal_mvcc_timestamp IS NOT NULL AS foo FROM t ORDER BY foo +---- +1 2 true + +# We shouldn't be able to insert into the MVCC column. +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +INSERT INTO t (x, crdb_internal_mvcc_timestamp) VALUES (1, 0) + +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +UPDATE t SET crdb_internal_mvcc_timestamp = 1.0 WHERE x = 1 + +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +UPSERT INTO t (x, crdb_internal_mvcc_timestamp) VALUES (1, 0) + +statement error pq: INSERT has more expressions than target columns, 4 expressions for 3 targets +INSERT INTO t VALUES (7, 8, 9, 1.0) + +# TODO (rohany): I'm not sure how to disable returning the system columns. The way +# that we build the returning set doesn't have an easy hook to disallow system cols. +# statement error pq: column \"crdb_internal_mvcc_timestamp\" does not exist +# INSERT INTO t VALUES (1, 2, 3) RETURNING crdb_internal_mvcc_timestamp + +# Ensure that we can't create columns that conflict with system column names. +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +CREATE TABLE bad (crdb_internal_mvcc_timestamp int) + +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +ALTER TABLE t ADD COLUMN crdb_internal_mvcc_timestamp INT + +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +ALTER TABLE t RENAME COLUMN x TO crdb_internal_mvcc_timestamp diff --git a/pkg/sql/logictest/testdata/logic_test/tpch_vec b/pkg/sql/logictest/testdata/logic_test/tpch_vec index f49972bf5b0f..bf7545daf328 100644 --- a/pkg/sql/logictest/testdata/logic_test/tpch_vec +++ b/pkg/sql/logictest/testdata/logic_test/tpch_vec @@ -1,3 +1,4 @@ + # LogicTest: local # Note that statistics are populated for TPCH Scale Factor 1. @@ -541,7 +542,7 @@ EXPLAIN (VEC) SELECT l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, sum └ *colexec.projMultFloat64Float64Op └ *colexec.projMinusFloat64ConstFloat64Op └ *colexec.selLEInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 2 query T @@ -555,24 +556,24 @@ EXPLAIN (VEC) SELECT s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_ └ *colexec.hashAggregator └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *colexec.mergeJoinInnerOp - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selEQBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selEQBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selSuffixBytesBytesConstOp └ *colexec.selEQInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 3 query T @@ -585,12 +586,12 @@ EXPLAIN (VEC) SELECT l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS reve └ *colexec.hashAggregator └ *rowexec.joinReader └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.hashJoiner ├ *colexec.selLTInt64Int64ConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 4 query T @@ -602,7 +603,7 @@ EXPLAIN (VEC) SELECT o_orderpriority, count(*) AS order_count FROM orders WHERE └ *colexec.hashAggregator └ *rowexec.joinReader └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 5 query T @@ -617,15 +618,15 @@ EXPLAIN (VEC) SELECT n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue └ *colexec.hashJoiner ├ *colexec.hashJoiner │ ├ *colexec.hashJoiner - │ │ ├ *colfetcher.colBatchScan + │ │ ├ *colfetcher.ColBatchScan │ │ └ *rowexec.joinReader │ │ └ *colexec.hashJoiner - │ │ ├ *colfetcher.colBatchScan + │ │ ├ *colfetcher.ColBatchScan │ │ └ *colexec.selEQBytesBytesConstOp - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *rowexec.indexJoiner - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 6 query T @@ -636,7 +637,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice * l_discount) AS revenue FROM lineitem └ *colexec.orderedAggregator └ *colexec.distinctChainOps └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 7 query T @@ -653,14 +654,14 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR └ *colexec.hashJoiner ├ *rowexec.joinReader │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *rowexec.joinReader │ └ *colexec.caseOp │ ├ *colexec.bufferOp │ │ └ *colexec.hashJoiner - │ │ ├ *colfetcher.colBatchScan - │ │ └ *colfetcher.colBatchScan + │ │ ├ *colfetcher.ColBatchScan + │ │ └ *colfetcher.ColBatchScan │ ├ *colexec.constBoolOp │ │ └ *colexec.andProjOp │ │ ├ *colexec.bufferOp @@ -673,7 +674,7 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR │ │ └ *colexec.projEQBytesBytesConstOp │ └ *colexec.constBoolOp │ └ *colexec.bufferOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 8 query T @@ -697,16 +698,16 @@ EXPLAIN (VEC) SELECT o_year, sum(CASE WHEN nation = 'BRAZIL' THEN volume ELSE 0 │ │ │ │ └ *colexec.hashJoiner │ │ │ │ ├ *colexec.selLEInt64Int64ConstOp │ │ │ │ │ └ *colexec.selGEInt64Int64ConstOp - │ │ │ │ │ └ *colfetcher.colBatchScan + │ │ │ │ │ └ *colfetcher.ColBatchScan │ │ │ │ └ *rowexec.joinReader │ │ │ │ └ *colexec.mergeJoinInnerOp - │ │ │ │ ├ *colfetcher.colBatchScan + │ │ │ │ ├ *colfetcher.ColBatchScan │ │ │ │ └ *colexec.selEQBytesBytesConstOp - │ │ │ │ └ *colfetcher.colBatchScan - │ │ │ └ *colfetcher.colBatchScan - │ │ └ *colfetcher.colBatchScan + │ │ │ │ └ *colfetcher.ColBatchScan + │ │ │ └ *colfetcher.ColBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.selEQBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.projEQBytesBytesConstOp │ └ *colexec.bufferOp └ *colexec.constFloat64Op @@ -723,12 +724,12 @@ EXPLAIN (VEC) SELECT nation, o_year, sum(amount) AS sum_profit FROM ( SELECT n_n └ *rowexec.joinReader └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 10 query T @@ -742,10 +743,10 @@ EXPLAIN (VEC) SELECT c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) └ *rowexec.joinReader └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.indexJoiner - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 11 query T @@ -762,7 +763,7 @@ EXPLAIN (VEC) SELECT ps_partkey, sum(ps_supplycost * ps_availqty::float) AS valu └ *rowexec.joinReader └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 12 query T @@ -774,7 +775,7 @@ EXPLAIN (VEC) SELECT l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' or o └ *colexec.hashAggregator └ *rowexec.joinReader └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 13 query T @@ -787,8 +788,8 @@ EXPLAIN (VEC) SELECT c_count, count(*) AS custdist FROM ( SELECT c_custkey, coun └ *colexec.hashAggregator └ *colexec.hashJoiner ├ *colexec.selNotRegexpBytesBytesConstOp - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 14 query T @@ -805,9 +806,9 @@ EXPLAIN (VEC) SELECT 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extended └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.indexJoiner - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.projMultFloat64Float64Op │ └ *colexec.projMinusFloat64ConstFloat64Op │ └ *colexec.projPrefixBytesBytesConstOp @@ -825,14 +826,14 @@ EXPLAIN (VEC) SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM s │ └ Node 1 └ *colexec.mergeJoinInnerOp - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.sortOp └ *colexec.selEQFloat64Float64Op └ *colexec.castOpNullAny └ *colexec.constNullOp └ *colexec.hashAggregator └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok DROP VIEW revenue0 @@ -848,13 +849,13 @@ EXPLAIN (VEC) SELECT p_brand, p_type, p_size, count(distinct ps_suppkey) AS supp └ *colexec.unorderedDistinct └ *colexec.hashJoiner ├ *colexec.mergeJoinLeftAntiOp - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selRegexpBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selectInOpInt64 └ *colexec.selNotPrefixBytesBytesConstOp └ *colexec.selNEBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 17 query T @@ -874,7 +875,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, par └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 18 query T @@ -886,15 +887,15 @@ EXPLAIN (VEC) SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, s └ *colexec.topKSorter └ *colexec.hashAggregator └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.hashJoiner ├ *colexec.mergeJoinLeftSemiOp - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selGTFloat64Float64ConstOp │ └ *colexec.orderedAggregator │ └ *colexec.distinctChainOps - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 19 query T @@ -911,9 +912,9 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice* (1 - l_discount)) AS revenue FROM line │ └ *colexec.hashJoiner │ ├ *colexec.selEQBytesBytesConstOp │ │ └ *colexec.selectInOpBytes - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.selGEInt64Int64ConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.constBoolOp │ └ *colexec.orProjOp │ ├ *colexec.bufferOp @@ -958,19 +959,19 @@ EXPLAIN (VEC) SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN └ *colexec.sortOp └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner │ ├ *colexec.selGTInt64Float64Op │ │ └ *colexec.projMultFloat64Float64ConstOp │ │ └ *colexec.hashAggregator │ │ └ *colexec.hashJoiner │ │ ├ *rowexec.indexJoiner - │ │ │ └ *colfetcher.colBatchScan - │ │ └ *colfetcher.colBatchScan + │ │ │ └ *colfetcher.ColBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.selPrefixBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 21 query T @@ -986,14 +987,14 @@ EXPLAIN (VEC) SELECT s_name, count(*) AS numwait FROM supplier, lineitem l1, ord ├ *rowexec.hashJoiner │ ├ *rowexec.mergeJoiner │ │ ├ *colexec.selGTInt64Int64Op - │ │ │ └ *colfetcher.colBatchScan + │ │ │ └ *colfetcher.ColBatchScan │ │ └ *colexec.selGTInt64Int64Op - │ │ └ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan └ *rowexec.joinReader └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 22 query T @@ -1011,4 +1012,4 @@ EXPLAIN (VEC) SELECT cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctba └ *colexec.substringInt64Int64Operator └ *colexec.constInt64Op └ *colexec.constInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize index 533deeb65779..c9194c62b4f8 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize +++ b/pkg/sql/logictest/testdata/logic_test/vectorize @@ -1235,3 +1235,22 @@ query T SELECT c0 FROM t47715 ORDER by c1 ---- 1819487610 + +# Ensure that the vectorized engine can plan and produce the MVCC system column. +statement ok +RESET vectorize; +CREATE TABLE mvcc (x INT PRIMARY KEY, y INT, z INT, INDEX i (z)); +INSERT INTO mvcc VALUES (1, 2, 3) + +statement ok +SET vectorize = experimental_always + +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM mvcc +---- +true + +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM mvcc@i +---- +true diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_agg b/pkg/sql/logictest/testdata/logic_test/vectorize_agg index 347b616e1cfb..764e0647aa34 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_agg +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_agg @@ -40,7 +40,7 @@ EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string G │ └ Node 1 └ *colexec.hashAggregator - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string @@ -49,7 +49,7 @@ EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string └ Node 1 └ *colexec.orderedAggregator └ *colexec.distinctChainOps - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET vectorize=experimental_always diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_local b/pkg/sql/logictest/testdata/logic_test/vectorize_local index 1ac8ce68531e..655484de86fd 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_local +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_local @@ -115,9 +115,9 @@ EXPLAIN (VEC) SELECT t46122_0.c0 FROM t46122_0, t46122_1 ---- │ └ Node 1 -└ *rowexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + └ *rowexec.hashJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Regression test for #46404 (rowexec.noopProcessor not implementing # execinfra.OpNode interface). @@ -129,11 +129,11 @@ EXPLAIN (VEC) SELECT stddev((t46404_1.c0 > ANY (0, 0))::INT) FROM t46404_0, t464 ---- │ └ Node 1 -└ *rowexec.hashAggregator - └ *rowexec.noopProcessor - └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + └ *rowexec.hashAggregator + └ *rowexec.noopProcessor + └ *colexec.hashJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan statement ok CREATE TABLE xyz ( @@ -150,8 +150,8 @@ EXPLAIN (VEC) SELECT * FROM xyz AS t1 FULL OUTER JOIN xyz AS t2 ON t1.x = t2.x A │ └ Node 1 └ *rowexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Verify that the vectorized engine is used (there is a mismatch between # argument type width and the result). @@ -162,7 +162,7 @@ EXPLAIN (VEC) SELECT max(c) FROM a └ Node 1 └ *colexec.orderedAggregator └ *colexec.distinctChainOps - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Verify that binary operations on integers of any width return INT8. statement ok @@ -187,7 +187,7 @@ EXPLAIN (VEC) SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 └ *colexec.projPlusInt64Int64Op └ *colexec.castOp └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 @@ -204,7 +204,7 @@ EXPLAIN (VEC) SELECT c.a FROM c JOIN d ON d.b = c.b │ └ Node 1 └ *rowexec.joinReader - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET vectorize = experimental_always diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_overloads b/pkg/sql/logictest/testdata/logic_test/vectorize_overloads index f07d249e61fa..347ed6822c66 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_overloads +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_overloads @@ -92,7 +92,7 @@ EXPLAIN (VEC) SELECT _inet & _inet FROM many_types │ └ Node 1 └ *colexec.projBitandDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _inet & _inet FROM many_types @@ -108,7 +108,7 @@ EXPLAIN (VEC) SELECT _inet - _int2 FROM many_types └ Node 1 └ *colexec.projMinusDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int2^_int4 FROM many_types @@ -118,7 +118,7 @@ EXPLAIN (VEC) SELECT _int2^_int4 FROM many_types └ *colexec.projPowInt64Int64Op └ *colexec.castOp └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int2^_int FROM many_types @@ -127,7 +127,7 @@ EXPLAIN (VEC) SELECT _int2^_int FROM many_types └ Node 1 └ *colexec.projPowInt64Int64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _float^_float FROM many_types @@ -135,7 +135,7 @@ EXPLAIN (VEC) SELECT _float^_float FROM many_types │ └ Node 1 └ *colexec.projPowFloat64Float64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _decimal^_int4 FROM many_types @@ -144,7 +144,7 @@ EXPLAIN (VEC) SELECT _decimal^_int4 FROM many_types └ Node 1 └ *colexec.projPowDecimalInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query R rowsort SELECT _float^_float FROM many_types @@ -173,7 +173,7 @@ EXPLAIN (VEC) SELECT _inet - 1 FROM many_types │ └ Node 1 └ *colexec.projMinusDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _inet - 1 FROM many_types @@ -189,7 +189,7 @@ EXPLAIN (VEC) SELECT _int4 + _inet FROM many_types └ Node 1 └ *colexec.projPlusDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _int4 + _inet FROM many_types @@ -204,7 +204,7 @@ EXPLAIN (VEC) SELECT 2 + _inet FROM many_types │ └ Node 1 └ *colexec.projPlusDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT 2 + _inet FROM many_types @@ -219,7 +219,7 @@ EXPLAIN (VEC) SELECT _time + _interval FROM many_types │ └ Node 1 └ *colexec.projPlusDatumIntervalOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _time + _interval FROM many_types @@ -234,7 +234,7 @@ EXPLAIN (VEC) SELECT _json - _int FROM many_types │ └ Node 1 └ *colexec.projMinusDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _json - _int FROM many_types @@ -249,7 +249,7 @@ EXPLAIN (VEC) SELECT _bytes || _bytes FROM many_types │ └ Node 1 └ *colexec.projConcatBytesBytesOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _bytes || _bytes FROM many_types @@ -264,7 +264,7 @@ EXPLAIN (VEC) SELECT _string || _string FROM many_types │ └ Node 1 └ *colexec.projConcatBytesBytesOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _string || _string FROM many_types @@ -279,7 +279,7 @@ EXPLAIN (VEC) SELECT _json || _json FROM many_types │ └ Node 1 └ *colexec.projConcatDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json || _json FROM many_types @@ -294,7 +294,7 @@ EXPLAIN (VEC) SELECT _varbit || _varbit FROM many_types │ └ Node 1 └ *colexec.projConcatDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _varbit || _varbit FROM many_types @@ -309,7 +309,7 @@ EXPLAIN (VEC) SELECT _int << 1 FROM many_types │ └ Node 1 └ *colexec.projLShiftInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int >> 1 FROM many_types @@ -317,7 +317,7 @@ EXPLAIN (VEC) SELECT _int >> 1 FROM many_types │ └ Node 1 └ *colexec.projRShiftInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I rowsort SELECT _int2 >> 1 FROM many_types @@ -376,7 +376,7 @@ EXPLAIN (VEC) SELECT _varbit << 4 FROM many_types │ └ Node 1 └ *colexec.projLShiftDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int2 FROM many_types @@ -385,7 +385,7 @@ EXPLAIN (VEC) SELECT _varbit << _int2 FROM many_types └ Node 1 └ *colexec.projLShiftDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int4 FROM many_types @@ -394,7 +394,7 @@ EXPLAIN (VEC) SELECT _varbit << _int4 FROM many_types └ Node 1 └ *colexec.projLShiftDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int FROM many_types @@ -402,7 +402,7 @@ EXPLAIN (VEC) SELECT _varbit << _int FROM many_types │ └ Node 1 └ *colexec.projLShiftDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T @@ -411,7 +411,7 @@ EXPLAIN (VEC) SELECT _varbit >> 4 FROM many_types │ └ Node 1 └ *colexec.projRShiftDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int2 FROM many_types @@ -420,7 +420,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int2 FROM many_types └ Node 1 └ *colexec.projRShiftDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int4 FROM many_types @@ -429,7 +429,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int4 FROM many_types └ Node 1 └ *colexec.projRShiftDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int FROM many_types @@ -437,7 +437,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int FROM many_types │ └ Node 1 └ *colexec.projRShiftDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _varbit >> 1 FROM many_types @@ -470,7 +470,7 @@ EXPLAIN (VEC) SELECT _json -> _int2 FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I rowsort SELECT _int2^_int FROM many_types WHERE _int2 < 10 AND _int < 10 @@ -505,7 +505,7 @@ EXPLAIN (VEC) SELECT _json -> _int4 FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op └ *colexec.castOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> _int4 FROM many_types @@ -521,7 +521,7 @@ EXPLAIN (VEC) SELECT _json -> _int FROM many_types │ └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> _int FROM many_types @@ -537,7 +537,7 @@ EXPLAIN (VEC) SELECT _json -> 2 FROM many_types │ └ Node 1 └ *colexec.projJSONFetchValDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> 2 FROM many_types @@ -554,7 +554,7 @@ EXPLAIN (VEC) SELECT _json -> 2 -> 'a' FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumBytesConstOp └ *colexec.projJSONFetchValDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> 2 -> 'a' FROM many_types @@ -585,7 +585,7 @@ EXPLAIN (VEC) SELECT '[2, "hi", {"b": ["bar", {"c": 4}]}]'::jsonb -> _int FROM m │ └ Node 1 └ *colexec.projJSONFetchValDatumConstInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT '[2, "hi", {"b": ["bar", {"c": 4}]}]'::jsonb -> _int FROM many_types @@ -604,7 +604,7 @@ EXPLAIN (VEC) SELECT B'11' <> _varbit FROM many_types │ └ Node 1 └ *colexec.projNEDatumDatumConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query B rowsort SELECT B'11' >= _varbit FROM many_types diff --git a/pkg/sql/opt/cat/column.go b/pkg/sql/opt/cat/column.go index 8a8168968da4..077767d95471 100644 --- a/pkg/sql/opt/cat/column.go +++ b/pkg/sql/opt/cat/column.go @@ -28,6 +28,11 @@ const ( // DeleteOnly columns are mutation columns that have to be updated only on // deletes and cannot be otherwise accessed. DeleteOnly + // SystemColumn columns are implicit columns that every physical table + // contains. These columns can only be read from and must not be included + // as part of mutations. These columns are synthesized entirely within the + // optimizer and are not part of the persisted table definition. + SystemColumn ) // Column is an interface to a table column, exposing only the information @@ -103,3 +108,9 @@ func IsMutationColumn(table Table, ord int) bool { kind := table.ColumnKind(ord) return kind == WriteOnly || kind == DeleteOnly } + +// IsSystemColumn is a convenience function that returns true if the column at +// the given ordinal position is a system column. +func IsSystemColumn(table Table, ord int) bool { + return table.ColumnKind(ord) == SystemColumn +} diff --git a/pkg/sql/opt/cat/index.go b/pkg/sql/opt/cat/index.go index a6f37acc592e..0673c469e053 100644 --- a/pkg/sql/opt/cat/index.go +++ b/pkg/sql/opt/cat/index.go @@ -52,9 +52,16 @@ type Index interface { // IsInverted returns true if this is an inverted index. IsInverted() bool + // ColumnCountNoSystemColumns returns the number of columns in the index, but + // excludes any system columns. These columns shouldn't be considered by + // statistics when computing the cost of scanning an index. + ColumnCountNoSystemColumns() int + // ColumnCount returns the number of columns in the index. This includes // columns that were part of the index definition (including the STORING - // clause), as well as implicitly added primary key columns. + // clause), as well as implicitly added primary key columns. This also the + // implicit system columns, which are placed after all physical columns in + // the table. ColumnCount() int // Predicate returns the partial index predicate expression and true if the diff --git a/pkg/sql/opt/cat/utils.go b/pkg/sql/opt/cat/utils.go index 61d967adcc18..c2065f7f575a 100644 --- a/pkg/sql/opt/cat/utils.go +++ b/pkg/sql/opt/cat/utils.go @@ -132,7 +132,7 @@ func FormatTable(cat Catalog, tab Table, tp treeprinter.Node) { var buf bytes.Buffer for i := 0; i < tab.ColumnCount(); i++ { buf.Reset() - formatColumn(tab.Column(i), IsMutationColumn(tab, i), &buf) + formatColumn(tab.Column(i), IsMutationColumn(tab, i), IsSystemColumn(tab, i), &buf) child.Child(buf.String()) } @@ -189,7 +189,7 @@ func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node) { buf.Reset() idxCol := idx.Column(i) - formatColumn(idxCol.Column, false /* isMutationCol */, &buf) + formatColumn(idxCol.Column, false /* isMutationCol */, IsSystemColumn(tab, idxCol.Ordinal), &buf) if idxCol.Descending { fmt.Fprintf(&buf, " desc") } @@ -284,7 +284,7 @@ func formatCatalogFKRef( ) } -func formatColumn(col Column, isMutationCol bool, buf *bytes.Buffer) { +func formatColumn(col Column, isMutationCol bool, isSystemCol bool, buf *bytes.Buffer) { fmt.Fprintf(buf, "%s %s", col.ColName(), col.DatumType()) if !col.IsNullable() { fmt.Fprintf(buf, " not null") @@ -301,6 +301,9 @@ func formatColumn(col Column, isMutationCol bool, buf *bytes.Buffer) { if isMutationCol { fmt.Fprintf(buf, " [mutation]") } + if isSystemCol { + fmt.Fprintf(buf, " [system]") + } } func formatFamily(family Family, buf *bytes.Buffer) { diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 755035016e46..831763192175 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -704,7 +704,8 @@ func mutationOutputColMap(mutation memo.RelExpr) opt.ColMap { ord := 0 for i, n := 0, tab.ColumnCount(); i < n; i++ { colID := private.Table.ColumnID(i) - if outCols.Contains(colID) { + // System columns should not be included in mutations. + if outCols.Contains(colID) && !cat.IsSystemColumn(tab, i) { colMap.Set(int(colID), ord) ord++ } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/aggregate b/pkg/sql/opt/exec/execbuilder/testdata/aggregate index a39cb95c5a7f..ad6177855d5f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/aggregate +++ b/pkg/sql/opt/exec/execbuilder/testdata/aggregate @@ -14,19 +14,19 @@ EXPLAIN (TYPES) SELECT min(1), max(1), count(NULL), sum_int(1), avg(1), sum(1), · distribution local · · · vectorized true · · group · · (min int, max int, count int, sum_int int, avg decimal, sum decimal, stddev decimal, variance decimal, bool_and bool, bool_or bool, xor_agg bytes) · - │ aggregate 0 min(column5) · · - │ aggregate 1 max(column5) · · - │ aggregate 2 count(column8) · · - │ aggregate 3 sum_int(column5) · · - │ aggregate 4 avg(column5) · · - │ aggregate 5 sum(column5) · · - │ aggregate 6 stddev(column5) · · - │ aggregate 7 variance(column5) · · - │ aggregate 8 bool_and(column15) · · - │ aggregate 9 bool_or(column17) · · - │ aggregate 10 xor_agg(column19) · · + │ aggregate 0 min(column6) · · + │ aggregate 1 max(column6) · · + │ aggregate 2 count(column9) · · + │ aggregate 3 sum_int(column6) · · + │ aggregate 4 avg(column6) · · + │ aggregate 5 sum(column6) · · + │ aggregate 6 stddev(column6) · · + │ aggregate 7 variance(column6) · · + │ aggregate 8 bool_and(column16) · · + │ aggregate 9 bool_or(column18) · · + │ aggregate 10 xor_agg(column20) · · │ scalar · · · - └── render · · (column5 int, column8 unknown, column15 bool, column17 bool, column19 bytes) · + └── render · · (column6 int, column9 unknown, column16 bool, column18 bool, column20 bytes) · │ render 0 (1)[int] · · │ render 1 (NULL)[unknown] · · │ render 2 (true)[bool] · · @@ -57,15 +57,15 @@ render · · (min int, max │ aggregate 0 min(v) · · │ aggregate 1 max(v) · · │ aggregate 2 count(v) · · - │ aggregate 3 sum_int(column8) · · + │ aggregate 3 sum_int(column9) · · │ aggregate 4 avg(v) · · │ aggregate 5 sum(v) · · │ aggregate 6 stddev(v) · · │ aggregate 7 variance(v) · · - │ aggregate 8 bool_and(column14) · · - │ aggregate 9 xor_agg(column16) · · + │ aggregate 8 bool_and(column15) · · + │ aggregate 9 xor_agg(column17) · · │ scalar · · · - └── render · · (column8 int, column14 bool, column16 bytes, v int) · + └── render · · (column9 int, column15 bool, column17 bytes, v int) · │ render 0 (1)[int] · · │ render 1 ((v)[int] = (1)[int])[bool] · · │ render 2 ((s)[string]::BYTES)[bytes] · · @@ -137,12 +137,12 @@ EXPLAIN (TYPES) SELECT count(*), k+v AS r FROM kv GROUP BY k+v · vectorized true · · render · · (count int, r int) · │ render 0 (count_rows)[int] · · - │ render 1 (column6)[int] · · - └── group · · (column6 int, count_rows int) · - │ aggregate 0 column6 · · + │ render 1 (column7)[int] · · + └── group · · (column7 int, count_rows int) · + │ aggregate 0 column7 · · │ aggregate 1 count_rows() · · - │ group by column6 · · - └── render · · (column6 int) · + │ group by column7 · · + └── render · · (column7 int) · │ render 0 ((k)[int] + (v)[int])[int] · · └── scan · · (k int, v int) · · table kv@primary · · @@ -216,11 +216,11 @@ EXPLAIN (VERBOSE) SELECT count(DISTINCT a.*) FROM kv a, kv b · distribution local · · · vectorized true · · group · · (count) · - │ aggregate 0 count(column9) · · + │ aggregate 0 count(column11) · · │ scalar · · · - └── distinct · · (column9) · - │ distinct on column9 · · - └── render · · (column9) · + └── distinct · · (column11) · + │ distinct on column11 · · + └── render · · (column11) · │ render 0 ((k, v, w, s) AS k, v, w, s) · · └── cross-join · · (k, v, w, s) · │ type cross · · @@ -624,10 +624,10 @@ render · · (v int, count int) │ order +count_rows · · └── group · · (v int, count int, count_rows int) · │ aggregate 0 v · · - │ aggregate 1 count(column5) · · + │ aggregate 1 count(column6) · · │ aggregate 2 count_rows() · · │ group by v · · - └── render · · (column5 unknown, v int) · + └── render · · (column6 unknown, v int) · │ render 0 (NULL)[unknown] · · │ render 1 (v)[int] · · └── scan · · (v int) · @@ -642,9 +642,9 @@ EXPLAIN (VERBOSE) SELECT * FROM (SELECT v, count(NULL) FROM kv GROUP BY v) WHERE · vectorized true · · group · · (v, count) · │ aggregate 0 v · · - │ aggregate 1 count(column5) · · + │ aggregate 1 count(column6) · · │ group by v · · - └── render · · (column5, v) · + └── render · · (column6, v) · │ render 0 NULL · · │ render 1 v · · └── scan · · (v) · @@ -672,10 +672,10 @@ render · · (cou │ render 1 max · · └── group · · (v, count, max) · │ aggregate 0 v · · - │ aggregate 1 count(column5) FILTER (WHERE column6) · · - │ aggregate 2 max(column6) FILTER (WHERE column6) · · + │ aggregate 1 count(column6) FILTER (WHERE column7) · · + │ aggregate 2 max(column7) FILTER (WHERE column7) · · │ group by v · · - └── render · · (column5, column6, v) · + └── render · · (column6, column7, v) · │ render 0 true · · │ render 1 k > 5 · · │ render 2 v · · @@ -692,9 +692,9 @@ render · · (cou │ render 0 count · · └── group · · (v, count) · │ aggregate 0 v · · - │ aggregate 1 count(column5) FILTER (WHERE column6) · · + │ aggregate 1 count(column6) FILTER (WHERE column7) · · │ group by v · · - └── render · · (column5, column6, v) · + └── render · · (column6, column7, v) · │ render 0 true · · │ render 1 k > 5 · · │ render 2 v · · @@ -781,9 +781,9 @@ EXPLAIN (TYPES) SELECT min(v+1) FROM opt_test · distribution local · · · vectorized true · · group · · (min int) · - │ aggregate 0 min(column3) · · + │ aggregate 0 min(column4) · · │ scalar · · · - └── render · · (column3 int) · + └── render · · (column4 int) · │ render 0 ((v)[int] + (1)[int])[int] · · └── scan · · (v int) · · table opt_test@primary · · @@ -965,11 +965,11 @@ EXPLAIN (TYPES) SELECT 1 a FROM kv GROUP BY v, w::DECIMAL HAVING w::DECIMAL > 1; · vectorized true · · render · · (a int) · │ render 0 (1)[int] · · - └── distinct · · (column5 decimal, v int) · - │ distinct on column5, v · · - └── filter · · (column5 decimal, v int) · - │ filter ((column5)[decimal] > (1)[decimal])[bool] · · - └── render · · (column5 decimal, v int) · + └── distinct · · (column6 decimal, v int) · + │ distinct on column6, v · · + └── filter · · (column6 decimal, v int) · + │ filter ((column6)[decimal] > (1)[decimal])[bool] · · + └── render · · (column6 decimal, v int) · │ render 0 ((w)[int]::DECIMAL)[decimal] · · │ render 1 (v)[int] · · └── scan · · (v int, w int) · @@ -987,11 +987,11 @@ EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @1 · vectorized true · · render · · (m) · │ render 0 min · · - └── group · · (column5, min) · - │ aggregate 0 column5 · · + └── group · · (column6, min) · + │ aggregate 0 column6 · · │ aggregate 1 min(a) · · - │ group by column5 · · - └── render · · (column5, a) · + │ group by column6 · · + └── render · · (column6, a) · │ render 0 a · · │ render 1 a · · └── scan · · (a) · @@ -1005,11 +1005,11 @@ EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @2 · vectorized true · · render · · (m) · │ render 0 min · · - └── group · · (column5, min) · - │ aggregate 0 column5 · · + └── group · · (column6, min) · + │ aggregate 0 column6 · · │ aggregate 1 min(a) · · - │ group by column5 · · - └── render · · (column5, a) · + │ group by column6 · · + └── render · · (column6, a) · │ render 0 b · · │ render 1 a · · └── scan · · (a, b) · @@ -1075,9 +1075,9 @@ EXPLAIN (VERBOSE) SELECT string_agg(s, ',') FROM (SELECT s FROM kv ORDER BY k) · distribution local · · · vectorized true · · group · · (string_agg) · - │ aggregate 0 string_agg(s, column5) · · + │ aggregate 0 string_agg(s, column6) · · │ scalar · · · - └── render · · (column5, k, s) +k + └── render · · (column6, k, s) +k │ render 0 ',' · · │ render 1 k · · │ render 2 s · · @@ -1132,9 +1132,9 @@ EXPLAIN (VERBOSE) SELECT string_agg(s, ', ') FROM kv · distribution local · · · vectorized true · · group · · (string_agg) · - │ aggregate 0 string_agg(s, column5) · · + │ aggregate 0 string_agg(s, column6) · · │ scalar · · · - └── render · · (column5, s) · + └── render · · (column6, s) · │ render 0 ', ' · · │ render 1 s · · └── scan · · (s) · @@ -1165,9 +1165,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(employee, column4) · · + │ aggregate 1 string_agg(employee, column5) · · │ group by company_id · · - └── render · · (column4, company_id, employee) · + └── render · · (column5, company_id, employee) · │ render 0 ',' · · │ render 1 company_id · · │ render 2 employee · · @@ -1192,9 +1192,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(column4, column5) · · + │ aggregate 1 string_agg(column5, column6) · · │ group by company_id · · - └── render · · (column4, column5, company_id) · + └── render · · (column5, column6, company_id) · │ render 0 employee::BYTES · · │ render 1 '\x2c' · · │ render 2 company_id · · @@ -1219,9 +1219,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(employee, column4) · · + │ aggregate 1 string_agg(employee, column5) · · │ group by company_id · · - └── render · · (column4, company_id, employee) · + └── render · · (column5, company_id, employee) · │ render 0 NULL · · │ render 1 company_id · · │ render 2 employee · · @@ -1246,9 +1246,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(column4, column5) · · + │ aggregate 1 string_agg(column5, column6) · · │ group by company_id · · - └── render · · (column4, column5, company_id) · + └── render · · (column5, column6, company_id) · │ render 0 employee::BYTES · · │ render 1 NULL · · │ render 2 company_id · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/catalog b/pkg/sql/opt/exec/execbuilder/testdata/catalog index 58e1228947e2..842ecef6e0f0 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/catalog +++ b/pkg/sql/opt/exec/execbuilder/testdata/catalog @@ -16,6 +16,7 @@ TABLE xyz ├── x int not null ├── y int ├── z int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── x int not null └── INDEX foo @@ -48,6 +49,7 @@ TABLE abcdef ├── e int as (a) stored ├── f int not null ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── CHECK (f > 2:::INT8) └── INDEX primary └── rowid int not null default (unique_rowid()) [hidden] @@ -82,6 +84,7 @@ TABLE uvwxy ├── w int ├── x int ├── y int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_u_v_w (u, v, w) ├── FAMILY fam_1_x (x) ├── FAMILY fam_2_y (y) @@ -111,6 +114,7 @@ TABLE child ├── p int ├── q int ├── r int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null └── CONSTRAINT fk FOREIGN KEY child (p, q, r) REFERENCES parent (p, q, r) @@ -124,6 +128,7 @@ TABLE parent ├── q int not null ├── r int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -149,6 +154,7 @@ TABLE child2 ├── p int ├── q int ├── r int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null └── CONSTRAINT fk FOREIGN KEY child2 (p, q, r) REFERENCES parent (p, q, r) MATCH FULL ON DELETE SET NULL @@ -162,6 +168,7 @@ TABLE parent ├── q int not null ├── r int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -196,6 +203,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM a TABLE a ├── a int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_other (a, other) └── INDEX primary ├── a int not null @@ -209,6 +217,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM ab TABLE ab ├── a int not null ├── b int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b (a, b) └── INDEX primary ├── a int not null @@ -228,6 +237,7 @@ TABLE abc ├── a int ├── b int ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b_c_k (a, b, c, k) ├── INDEX primary │ └── k int not null @@ -248,6 +258,7 @@ TABLE abx ├── a int not null ├── b int not null ├── x int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b_x (a, b, x) └── INDEX primary ├── a int not null diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg index 316b14d60733..196c4d902cef 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg @@ -329,14 +329,14 @@ query T EXPLAIN (opt,verbose) SELECT b, count(*) FROM data2 WHERE a=1 GROUP BY b ---- group-by - ├── columns: b:2 count:3 + ├── columns: b:2 count:4 ├── grouping columns: b:2 ├── internal-ordering: +2 opt(1) ├── stats: [rows=9.5617925, distinct(2)=9.5617925, null(2)=0] ├── cost: 11.1256179 ├── key: (2) - ├── fd: (2)-->(3) - ├── prune: (3) + ├── fd: (2)-->(4) + ├── prune: (4) ├── scan data2 │ ├── columns: a:1 b:2 │ ├── constraint: /1/2: [/1 - /1] @@ -348,7 +348,7 @@ group-by │ ├── prune: (2) │ └── interesting orderings: (+1,+2) (+2,+1) └── aggregations - └── count-rows [as=count_rows:3] + └── count-rows [as=count_rows:4] query TTTTT EXPLAIN (verbose) SELECT b, count(*), corr(a, b) FROM data2 WHERE a=1 GROUP BY b diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 6319cd95262c..65c8fb78e5ba 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -907,6 +907,7 @@ TABLE tc ├── a int ├── b int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] └── INDEX c @@ -924,6 +925,7 @@ TABLE tc ├── a int ├── b int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] └── INDEX c @@ -932,33 +934,34 @@ TABLE tc TABLE t ├── k int not null ├── v int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary └── k int not null inner-join (hash) - ├── columns: a:1 b:2 k:4 v:5 + ├── columns: a:1 b:2 k:5 v:6 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(4)=99, null(4)=0] + ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] ├── cost: 2219.96 - ├── fd: (4)-->(5), (1)==(4), (4)==(1) - ├── prune: (2,5) + ├── fd: (5)-->(6), (1)==(5), (5)==(1) + ├── prune: (2,6) ├── scan tc │ ├── columns: a:1 b:2 │ ├── stats: [rows=1000, distinct(1)=100, null(1)=10] │ ├── cost: 1100.02 │ ├── prune: (1,2) │ ├── interesting orderings: (+1) - │ └── unfiltered-cols: (1-3) + │ └── unfiltered-cols: (1-4) ├── scan t - │ ├── columns: k:4 v:5 - │ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0] + │ ├── columns: k:5 v:6 + │ ├── stats: [rows=1000, distinct(5)=1000, null(5)=0] │ ├── cost: 1080.02 - │ ├── key: (4) - │ ├── fd: (4)-->(5) - │ ├── prune: (4,5) - │ ├── interesting orderings: (+4) - │ └── unfiltered-cols: (4,5) + │ ├── key: (5) + │ ├── fd: (5)-->(6) + │ ├── prune: (5,6) + │ ├── interesting orderings: (+5) + │ └── unfiltered-cols: (5-7) └── filters - └── k:4 = a:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── k:5 = a:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] query T EXPLAIN (OPT) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b @@ -976,21 +979,21 @@ query T EXPLAIN (OPT, VERBOSE) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort - ├── columns: a:1 b:2 [hidden: column4:4] + ├── columns: a:1 b:2 [hidden: column5:5] ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.26548 - ├── fd: (1,2)-->(4) - ├── ordering: +4 - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── ordering: +5 + ├── prune: (1,2,5) ├── interesting orderings: (+1) └── project - ├── columns: column4:4 a:1 b:2 + ├── columns: column5:5 a:1 b:2 ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.71667 - ├── fd: (1,2)-->(4) - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── prune: (1,2,5) ├── interesting orderings: (+1) ├── select │ ├── columns: a:1 b:2 @@ -1007,27 +1010,27 @@ sort │ └── filters │ └── (a:1 + (b:2 * 2)) > 1 [outer=(1,2), immutable] └── projections - └── a:1 * b:2 [as=column4:4, outer=(1,2), immutable] + └── a:1 * b:2 [as=column5:5, outer=(1,2), immutable] query T EXPLAIN (OPT, TYPES) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort - ├── columns: a:1(int) b:2(int) [hidden: column4:4(int)] + ├── columns: a:1(int) b:2(int) [hidden: column5:5(int)] ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.26548 - ├── fd: (1,2)-->(4) - ├── ordering: +4 - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── ordering: +5 + ├── prune: (1,2,5) ├── interesting orderings: (+1) └── project - ├── columns: column4:4(int) a:1(int) b:2(int) + ├── columns: column5:5(int) a:1(int) b:2(int) ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.71667 - ├── fd: (1,2)-->(4) - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── prune: (1,2,5) ├── interesting orderings: (+1) ├── select │ ├── columns: a:1(int) b:2(int) @@ -1050,7 +1053,7 @@ sort │ │ └── const: 2 [type=int] │ └── const: 1 [type=int] └── projections - └── mult [as=column4:4, type=int, outer=(1,2), immutable] + └── mult [as=column5:5, type=int, outer=(1,2), immutable] ├── variable: a:1 [type=int] └── variable: b:2 [type=int] diff --git a/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index b/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index index cdee4c5eca02..91bcb6c5e310 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index @@ -16,11 +16,11 @@ count · · │ into sharded_primary(crdb_internal_a_shard_11, a) · · │ strategy inserter · · │ auto commit · · · - └── render · · (column4, column1, check1) · - │ render 0 column4 · · + └── render · · (column5, column1, check1) · + │ render 0 column5 · · │ render 1 column1 · · - │ render 2 column4 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) · · - └── render · · (column4, column1) · + │ render 2 column5 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) · · + └── render · · (column5, column1) · │ render 0 mod(fnv32(COALESCE(column1::STRING, '')), 11) · · │ render 1 column1 · · └── values · · (column1) · @@ -41,12 +41,12 @@ count · · │ into sharded_secondary(a, crdb_internal_a_shard_12, rowid) · · │ strategy inserter · · │ auto commit · · · - └── render · · (column1, column6, column5, check1) · + └── render · · (column1, column7, column6, check1) · │ render 0 column1 · · - │ render 1 column6 · · - │ render 2 column5 · · - │ render 3 column6 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11) · · - └── render · · (column6, column5, column1) · + │ render 1 column7 · · + │ render 2 column6 · · + │ render 3 column7 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11) · · + └── render · · (column7, column6, column1) · │ render 0 mod(fnv32(COALESCE(column1::STRING, '')), 12) · · │ render 1 unique_rowid() · · │ render 2 column1 · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/insert b/pkg/sql/opt/exec/execbuilder/testdata/insert index 878bae16f710..071ef7c1b116 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/insert +++ b/pkg/sql/opt/exec/execbuilder/testdata/insert @@ -424,29 +424,29 @@ query TTTTT EXPLAIN (VERBOSE) INSERT INTO insert_t (SELECT length(k), 2 FROM kv ORDER BY k || v LIMIT 10) RETURNING x+v ---- -· distribution local · · -· vectorized false · · -render · · ("?column?") · - │ render 0 x + v · · - └── run · · (x, v, rowid[hidden]) · - └── insert · · (x, v, rowid[hidden]) · - │ into insert_t(x, v, rowid) · · - │ strategy inserter · · - └── render · · (length, "?column?", column9) · - │ render 0 length · · - │ render 1 "?column?" · · - │ render 2 unique_rowid() · · - └── limit · · (length, "?column?", column8) +column8 - │ count 10 · · - └── sort · · (length, "?column?", column8) +column8 - │ order +column8 · · - └── render · · (length, "?column?", column8) · - │ render 0 length(k) · · - │ render 1 2 · · - │ render 2 k::STRING || v::STRING · · - └── scan · · (k, v) · -· table kv@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +render · · ("?column?") · + │ render 0 x + v · · + └── run · · (x, v, rowid[hidden]) · + └── insert · · (x, v, rowid[hidden]) · + │ into insert_t(x, v, rowid) · · + │ strategy inserter · · + └── render · · (length, "?column?", column11) · + │ render 0 length · · + │ render 1 "?column?" · · + │ render 2 unique_rowid() · · + └── limit · · (length, "?column?", column10) +column10 + │ count 10 · · + └── sort · · (length, "?column?", column10) +column10 + │ order +column10 · · + └── render · · (length, "?column?", column10) · + │ render 0 length(k) · · + │ render 1 2 · · + │ render 2 k::STRING || v::STRING · · + └── scan · · (k, v) · +· table kv@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Insert rows into table during schema changes. @@ -511,34 +511,34 @@ CREATE TABLE xyz (x INT, y INT, z INT) query TTTTT EXPLAIN (VERBOSE) SELECT * FROM [INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z] ORDER BY z ---- -· distribution local · · -· vectorized false · · -root · · (z) +z - ├── sort · · (z) +z - │ │ order +z · · - │ └── scan buffer node · · (z) · - │ label buffer 1 · · - └── subquery · · · · - │ id @S1 · · - │ original sql INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · - │ exec mode all rows · · - └── buffer node · · (z) · - │ label buffer 1 · · - └── spool · · (z) · - └── render · · (z) · - │ render 0 z · · - └── run · · (z, rowid[hidden]) · - └── insert · · (z, rowid[hidden]) · - │ into xyz(x, y, z, rowid) · · - │ strategy inserter · · - └── render · · (a, b, c, column9) · - │ render 0 a · · - │ render 1 b · · - │ render 2 c · · - │ render 3 unique_rowid() · · - └── scan · · (a, b, c) · -· table abc@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +root · · (z) +z + ├── sort · · (z) +z + │ │ order +z · · + │ └── scan buffer node · · (z) · + │ label buffer 1 · · + └── subquery · · · · + │ id @S1 · · + │ original sql INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · + │ exec mode all rows · · + └── buffer node · · (z) · + │ label buffer 1 · · + └── spool · · (z) · + └── render · · (z) · + │ render 0 z · · + └── run · · (z, rowid[hidden]) · + └── insert · · (z, rowid[hidden]) · + │ into xyz(x, y, z, rowid) · · + │ strategy inserter · · + └── render · · (a, b, c, column11) · + │ render 0 a · · + │ render 1 b · · + │ render 2 c · · + │ render 3 unique_rowid() · · + └── scan · · (a, b, c) · +· table abc@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Regression for #35364. This tests behavior that is different between the CBO diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index 3183aa98229a..57c95e5406d1 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -487,10 +487,10 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type inner · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · @@ -541,10 +541,10 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type full outer · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · @@ -565,12 +565,12 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── filter · · (column6, a, b, n, sq) · + └── filter · · (column8, a, b, n, sq) · │ filter (b % 2) != (sq % 2) · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type full outer · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join index 67fcfad192fc..664f80560bde 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join @@ -1486,13 +1486,13 @@ EXPLAIN (VERBOSE) · vectorized true · · render · · (pk) · │ render 0 pk · · - └── lookup-join · · ("project_const_col_@13", pk, col0, col3) · + └── lookup-join · · ("project_const_col_@14", pk, col0, col3) · │ table tab4@tab4_col3_col4_key · · │ type semi · · - │ equality (col0, project_const_col_@13) = (col3, col4) · · + │ equality (col0, project_const_col_@14) = (col3, col4) · · │ equality cols are key · · · │ parallel · · · - └── render · · ("project_const_col_@13", pk, col0, col3) · + └── render · · ("project_const_col_@14", pk, col0, col3) · │ render 0 495.6 · · │ render 1 pk · · │ render 2 col0 · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/mvcc b/pkg/sql/opt/exec/execbuilder/testdata/mvcc new file mode 100644 index 000000000000..7ccc1ba1897f --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/mvcc @@ -0,0 +1,58 @@ +# LogicTest: local + +# Create a table with different families. +statement ok +CREATE TABLE t (x INT PRIMARY KEY, y INT, z INT, FAMILY (x), FAMILY (y), FAMILY (z)) + +# When doing a lookup where we could split families, but the MVCC column is +# requested, we shouldn't split the family. +query TTT +EXPLAIN SELECT z FROM t WHERE x = 1 +---- +· distribution local +· vectorized true +render · · + └── scan · · +· table t@primary +· spans /1/0-/1/1 /1/2/1-/1/2/2 + +query TTT +EXPLAIN SELECT crdb_internal_mvcc_timestamp, z FROM t WHERE x = 1 +---- +· distribution local +· vectorized true +render · · + └── scan · · +· table t@primary +· spans /1-/1/# + +# Ensure that the presence of mutation columns doesn't affect accessing system +# columns. +statement ok +BEGIN; +ALTER TABLE t ADD COLUMN w INT + +query TTT +EXPLAIN SELECT x, crdb_internal_mvcc_timestamp FROM t +---- +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN + +statement ok +ROLLBACK + +# Test that we can access system columns by ID. +let $t_id +SELECT id FROM system.namespace WHERE name = 't' + +query TTT +EXPLAIN SELECT * FROM [$t_id(4294967295) AS _] +---- +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN diff --git a/pkg/sql/opt/exec/execbuilder/testdata/orderby b/pkg/sql/opt/exec/execbuilder/testdata/orderby index 4ce380037091..0aee9c4cea9c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/orderby +++ b/pkg/sql/opt/exec/execbuilder/testdata/orderby @@ -690,9 +690,9 @@ EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @1 render · · (b, a) · │ render 0 b · · │ render 1 a · · - └── sort · · (column4, a, b) +a + └── sort · · (column5, a, b) +a │ order +a · · - └── render · · (column4, a, b) · + └── render · · (column5, a, b) · │ render 0 a · · │ render 1 a · · │ render 2 b · · @@ -708,9 +708,9 @@ EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @2 render · · (b, a) · │ render 0 b · · │ render 1 a · · - └── sort · · (column4, a, b) +b + └── sort · · (column5, a, b) +b │ order +b · · - └── render · · (column4, a, b) · + └── render · · (column5, a, b) · │ render 0 b · · │ render 1 a · · │ render 2 b · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select_for_update b/pkg/sql/opt/exec/execbuilder/testdata/select_for_update index 8a750915fc16..a8f5b9d691d3 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select_for_update +++ b/pkg/sql/opt/exec/execbuilder/testdata/select_for_update @@ -525,78 +525,40 @@ root · · query TTT EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE ---- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - │ locking strength for update - └── scan · · -· table t@primary -· spans FULL SCAN +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN +· locking strength for update query TTT EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE) ---- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - └── scan · · -· table t@primary -· spans FULL SCAN -· locking strength for update +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN query TTT EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE OF t ---- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - │ locking strength for update - └── scan · · -· table t@primary -· spans FULL SCAN +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN +· locking strength for update query TTT EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE OF t) ---- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - └── scan · · -· table t@primary -· spans FULL SCAN -· locking strength for update +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN # ------------------------------------------------------------------------------ # Tests with common-table expressions. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/srfs b/pkg/sql/opt/exec/execbuilder/testdata/srfs index 5d5377e4e5dd..ba9aab9cf259 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/srfs +++ b/pkg/sql/opt/exec/execbuilder/testdata/srfs @@ -243,17 +243,17 @@ render · · │ render 0 id · · │ render 1 data · · │ render 2 "?column?" · · - └── hash-join · · (column8, id, data, column9, "?column?") · + └── hash-join · · (column10, id, data, column11, "?column?") · │ type left outer · · - │ equality (column8) = (column9) · · - ├── render · · (column8, id, data) · + │ equality (column10) = (column11) · · + ├── render · · (column10, id, data) · │ │ render 0 data->>'name' · · │ │ render 1 id · · │ │ render 2 data · · │ └── scan · · (id, data) · │ table groups@primary · · │ spans FULL SCAN · · - └── render · · (column9, "?column?") · + └── render · · (column11, "?column?") · │ render 0 data->>'name' · · │ render 1 data->'members' · · └── scan · · (data) · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/subquery b/pkg/sql/opt/exec/execbuilder/testdata/subquery index 7a81fef3b832..f8bf340d57eb 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/subquery +++ b/pkg/sql/opt/exec/execbuilder/testdata/subquery @@ -184,12 +184,12 @@ EXPLAIN (VERBOSE) SELECT * FROM a WHERE EXISTS(SELECT * FROM b WHERE b.x-1 = a.x · vectorized true · · hash-join · · (x, y) · │ type semi · · - │ equality (x) = (column5) · · + │ equality (x) = (column7) · · │ left cols are key · · · ├── scan · · (x, y) · │ table a@primary · · │ spans FULL SCAN · · - └── render · · (column5) · + └── render · · (column7) · │ render 0 x - 1 · · └── scan · · (x) · · table b@primary · · @@ -220,12 +220,12 @@ EXPLAIN (VERBOSE) SELECT * FROM b WHERE NOT EXISTS(SELECT * FROM a WHERE x-1 = b · vectorized true · · hash-join · · (x, z) · │ type anti · · - │ equality (x) = (column5) · · + │ equality (x) = (column7) · · │ left cols are key · · · ├── scan · · (x, z) · │ table b@primary · · │ spans FULL SCAN · · - └── render · · (column5) · + └── render · · (column7) · │ render 0 x - 1 · · └── scan · · (x) · · table a@primary · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 0483e5e65406..36248dc845be 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -96,7 +96,7 @@ count · · │ auto commit · └── render · · │ render 0 k - │ render 1 column5 + │ render 1 column7 │ render 2 k └── lookup-join · · │ table kv@primary @@ -147,24 +147,24 @@ count · · │ auto commit · └── render · · │ render 0 column1 - │ render 1 column6 - │ render 2 column7 - │ render 3 column8 + │ render 1 column7 + │ render 2 column8 + │ render 3 column9 │ render 4 a │ render 5 b │ render 6 c │ render 7 d - │ render 8 column6 - │ render 9 column7 - │ render 10 column8 + │ render 8 column7 + │ render 9 column8 + │ render 10 column9 │ render 11 a │ render 12 check1 └── render · · - │ render 0 column7 > 0 + │ render 0 column8 > 0 │ render 1 column1 - │ render 2 column6 - │ render 3 column7 - │ render 4 column8 + │ render 2 column7 + │ render 3 column8 + │ render 4 column9 │ render 5 a │ render 6 b │ render 7 c @@ -199,12 +199,12 @@ run · · │ auto commit · └── render · · │ render 0 column1 - │ render 1 column6 - │ render 2 column7 - │ render 3 column8 - │ render 4 column6 - │ render 5 column7 - │ render 6 column8 + │ render 1 column7 + │ render 2 column8 + │ render 3 column9 + │ render 4 column7 + │ render 5 column8 + │ render 6 column9 │ render 7 check1 └── values · · · size 5 columns, 1 row @@ -349,42 +349,42 @@ CREATE TABLE xyz (x INT, y INT, z INT) query TTTTT EXPLAIN (VERBOSE) SELECT * FROM [UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z] ORDER BY z ---- -· distribution local · · -· vectorized false · · -root · · (z) +z - ├── sort · · (z) +z - │ │ order +z · · - │ └── scan buffer node · · (z) · - │ label buffer 1 · · - └── subquery · · · · - │ id @S1 · · - │ original sql UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · - │ exec mode all rows · · - └── buffer node · · (z) · - │ label buffer 1 · · - └── spool · · (z) · - └── render · · (z) · - │ render 0 z · · - └── run · · (z, rowid[hidden]) · - └── upsert · · (z, rowid[hidden]) · - │ into xyz(x, y, z, rowid) · · - │ strategy opt upserter · · - └── render · · (a, b, c, column9, a, b, c) · - │ render 0 a · · - │ render 1 b · · - │ render 2 c · · - │ render 3 column9 · · - │ render 4 a · · - │ render 5 b · · - │ render 6 c · · - └── render · · (column9, a, b, c) · - │ render 0 unique_rowid() · · - │ render 1 a · · - │ render 2 b · · - │ render 3 c · · - └── scan · · (a, b, c) · -· table abc@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +root · · (z) +z + ├── sort · · (z) +z + │ │ order +z · · + │ └── scan buffer node · · (z) · + │ label buffer 1 · · + └── subquery · · · · + │ id @S1 · · + │ original sql UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · + │ exec mode all rows · · + └── buffer node · · (z) · + │ label buffer 1 · · + └── spool · · (z) · + └── render · · (z) · + │ render 0 z · · + └── run · · (z, rowid[hidden]) · + └── upsert · · (z, rowid[hidden]) · + │ into xyz(x, y, z, rowid) · · + │ strategy opt upserter · · + └── render · · (a, b, c, column11, a, b, c) · + │ render 0 a · · + │ render 1 b · · + │ render 2 c · · + │ render 3 column11 · · + │ render 4 a · · + │ render 5 b · · + │ render 6 c · · + └── render · · (column11, a, b, c) · + │ render 0 unique_rowid() · · + │ render 1 a · · + │ render 2 b · · + │ render 3 c · · + └── scan · · (a, b, c) · +· table abc@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Regression for #35364. This tests behavior that is different between the CBO diff --git a/pkg/sql/opt/exec/execbuilder/testdata/with b/pkg/sql/opt/exec/execbuilder/testdata/with index 52c98a448ea4..07757f8f9974 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/with +++ b/pkg/sql/opt/exec/execbuilder/testdata/with @@ -63,7 +63,7 @@ root · · └── insert · · (a, rowid[hidden]) · │ into x(a, rowid) · · │ strategy inserter · · - └── values · · (column1, column4) · + └── values · · (column1, column5) · · size 2 columns, 1 row · · · row 0, expr 0 1 · · · row 0, expr 1 unique_rowid() · · diff --git a/pkg/sql/opt/memo/check_expr.go b/pkg/sql/opt/memo/check_expr.go index c17b9d08d0e9..1e5630757722 100644 --- a/pkg/sql/opt/memo/check_expr.go +++ b/pkg/sql/opt/memo/check_expr.go @@ -183,6 +183,9 @@ func (m *Memo) CheckExpr(e opt.Expr) { if tab.ColumnKind(i) != cat.DeleteOnly && t.InsertCols[i] == 0 { panic(errors.AssertionFailedf("insert values not provided for all table columns")) } + if cat.IsSystemColumn(tab, i) && t.InsertCols[i] != 0 { + panic(errors.AssertionFailedf("system column found in insertion columns")) + } } m.checkMutationExpr(t, &t.MutationPrivate) diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index e24784d89047..8e97e363b1b1 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -184,6 +184,16 @@ func neededMutationFetchCols( } } + // System columns might have been added into the needed fetch col set from the + // primary index. These columns are implicit and should not be part of the set + // of fetch columns, so we remove them here. + table := tabMeta.Table + for i := 0; i < table.ColumnCount(); i++ { + if cat.IsSystemColumn(table, i) { + cols.Remove(tabMeta.MetaID.ColumnID(i)) + } + } + return cols } diff --git a/pkg/sql/opt/optbuilder/fk_cascade.go b/pkg/sql/opt/optbuilder/fk_cascade.go index 2b7c8d6c5edb..e1ac1c67d3a6 100644 --- a/pkg/sql/opt/optbuilder/fk_cascade.go +++ b/pkg/sql/opt/optbuilder/fk_cascade.go @@ -112,7 +112,7 @@ func (cb *onDeleteCascadeBuilder) Build( ) // Set list of columns that will be fetched by the input expression. - for i := range mb.outScope.cols { + for i := range mb.fetchOrds { mb.fetchOrds[i] = scopeOrdinal(i) } mb.buildDelete(nil /* returning */) @@ -222,7 +222,7 @@ func (cb *onDeleteSetBuilder) Build( ) // Set list of columns that will be fetched by the input expression. - for i := range mb.outScope.cols { + for i := range mb.fetchOrds { mb.fetchOrds[i] = scopeOrdinal(i) } // Add target columns. @@ -440,11 +440,10 @@ func (cb *onUpdateCascadeBuilder) Build( // The scope created by b.buildUpdateCascadeMutationInput has the table // columns, followed by the old FK values, followed by the new FK values. numFKCols := fk.ColumnCount() - tableScopeCols := mb.outScope.cols[:len(mb.outScope.cols)-2*numFKCols] newValScopeCols := mb.outScope.cols[len(mb.outScope.cols)-numFKCols:] // Set list of columns that will be fetched by the input expression. - for i := range tableScopeCols { + for i := range mb.fetchOrds { mb.fetchOrds[i] = scopeOrdinal(i) } // Add target columns. diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index 88a73bb4d673..278f53a92473 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -371,6 +371,10 @@ func (mb *mutationBuilder) needExistingRows() bool { // #1: Don't consider key columns. continue } + if cat.IsSystemColumn(mb.tab, i) { + // #2: Don't consider system columns. + continue + } insertColID := mb.insertColID(i) if insertColID == 0 { // #2: Non-key column does not have insert value specified. @@ -513,8 +517,8 @@ func (mb *mutationBuilder) addTargetTableColsForInsert(maxCols int) { // the SQL user. numCols := 0 for i, n := 0, mb.tab.ColumnCount(); i < n && numCols < maxCols; i++ { - // Skip mutation or hidden columns. - if cat.IsMutationColumn(mb.tab, i) || mb.tab.Column(i).IsHidden() { + // Skip mutation, hidden or system columns. + if cat.IsMutationColumn(mb.tab, i) || mb.tab.Column(i).IsHidden() || cat.IsSystemColumn(mb.tab, i) { continue } @@ -561,7 +565,7 @@ func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.S } else { desiredTypes = make([]*types.T, 0, mb.tab.ColumnCount()) for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if !cat.IsMutationColumn(mb.tab, i) { + if !cat.IsMutationColumn(mb.tab, i) && !cat.IsSystemColumn(mb.tab, i) { tabCol := mb.tab.Column(i) if !tabCol.IsHidden() { desiredTypes = append(desiredTypes, tabCol.DatumType()) @@ -822,7 +826,7 @@ func (mb *mutationBuilder) buildInputForUpsert( mb.canaryColID = canaryScopeCol.id // Set fetchOrds to point to the scope columns created for the fetch values. - for i := range fetchScope.cols { + for i := range mb.fetchOrds { // Fetch columns come after insert columns. mb.fetchOrds[i] = scopeOrdinal(len(mb.outScope.cols) + i) } @@ -911,9 +915,9 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { copy(mb.updateOrds, mb.insertOrds) } - // Never update mutation columns. + // Never update mutation or system columns. for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if cat.IsMutationColumn(mb.tab, i) { + if cat.IsMutationColumn(mb.tab, i) || cat.IsSystemColumn(mb.tab, i) { mb.updateOrds[i] = -1 } } @@ -988,6 +992,11 @@ func (mb *mutationBuilder) projectUpsertColumns() { continue } + // Skip system columns. + if cat.IsSystemColumn(mb.tab, i) { + continue + } + // Generate CASE that toggles between insert and update column. caseExpr := mb.b.factory.ConstructCase( memo.TrueSingleton, @@ -1052,15 +1061,15 @@ func (mb *mutationBuilder) ensureUniqueConflictCols(conflictOrds util.FastIntSet } // mapPublicColumnNamesToOrdinals returns the set of ordinal positions within -// the target table that correspond to the given names. Mutation columns are -// ignored. +// the target table that correspond to the given names. Mutation and system +// columns are ignored. func (mb *mutationBuilder) mapPublicColumnNamesToOrdinals(names tree.NameList) util.FastIntSet { var ords util.FastIntSet for _, name := range names { found := false for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { tabCol := mb.tab.Column(i) - if tabCol.ColName() == name && !cat.IsMutationColumn(mb.tab, i) { + if tabCol.ColName() == name && !cat.IsMutationColumn(mb.tab, i) && !cat.IsSystemColumn(mb.tab, i) { ords.Add(i) found = true break diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 238b4ea4d538..97efb4b3c7e7 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -263,7 +263,6 @@ func (mb *mutationBuilder) buildInputForUpdate( ) fromClausePresent := len(from) > 0 - numCols := len(mb.outScope.cols) // If there is a FROM clause present, we must join all the tables // together with the table being updated. @@ -328,7 +327,7 @@ func (mb *mutationBuilder) buildInputForUpdate( } // Set list of columns that will be fetched by the input expression. - for i := 0; i < numCols; i++ { + for i := 0; i < len(mb.fetchOrds); i++ { mb.fetchOrds[i] = scopeOrdinal(i) } } @@ -389,7 +388,7 @@ func (mb *mutationBuilder) buildInputForDelete( mb.outScope = projectionsScope // Set list of columns that will be fetched by the input expression. - for i := range mb.outScope.cols { + for i := range mb.fetchOrds { mb.fetchOrds[i] = scopeOrdinal(i) } } @@ -401,6 +400,10 @@ func (mb *mutationBuilder) addTargetColsByName(names tree.NameList) { // Determine the ordinal position of the named column in the table and // add it as a target column. if ord := findPublicTableColumnByName(mb.tab, name); ord != -1 { + // System columns are invalid target columns. + if cat.IsSystemColumn(mb.tab, ord) { + panic(pgerror.Newf(pgcode.InvalidColumnReference, "cannot modify system column %q", name)) + } mb.addTargetCol(ord) continue } @@ -557,6 +560,11 @@ func (mb *mutationBuilder) addSynthesizedCols( continue } + // Skip system columns. + if cat.IsSystemColumn(mb.tab, i) { + continue + } + // Invoke addCol to determine whether column should be added. if !addCol(i) { continue @@ -831,8 +839,8 @@ func (mb *mutationBuilder) makeMutationPrivate(needResults bool) *memo.MutationP if needResults { private.ReturnCols = make(opt.ColList, mb.tab.ColumnCount()) for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if cat.IsMutationColumn(mb.tab, i) { - // Only non-mutation columns are output columns. + if cat.IsMutationColumn(mb.tab, i) || cat.IsSystemColumn(mb.tab, i) { + // Only non-mutation and non-system columns are output columns. continue } scopeOrd := mb.mapToReturnScopeOrd(i) diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index 7b9784f6e5a5..4f18e11f4ca1 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -410,12 +410,13 @@ func (tt *Table) addColumn(def *tree.ColumnTableDef) { func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) *Index { idx := &Index{ - IdxName: tt.makeIndexName(def.Name, typ), - Unique: typ != nonUniqueIndex, - Inverted: def.Inverted, - IdxZone: &zonepb.ZoneConfig{}, - table: tt, - partitionBy: def.PartitionBy, + IdxName: tt.makeIndexName(def.Name, typ), + Unique: typ != nonUniqueIndex, + Inverted: def.Inverted, + IdxZone: &zonepb.ZoneConfig{}, + table: tt, + partitionBy: def.PartitionBy, + numSystemColumns: 0, } // Look for name suffixes indicating this is a mutation index. @@ -488,6 +489,8 @@ func (tt *Table) addIndex(def *tree.IndexTableDef, typ indexType) *Index { if len(tt.Indexes) != 0 { panic("primary index should always be 0th index") } + // The primary index stores some system columns, so remember that here. + idx.numSystemColumns = tt.DeletableAndSystemColumnCount() - tt.DeletableColumnCount() idx.ordinal = len(tt.Indexes) tt.Indexes = append(tt.Indexes, idx) return idx diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index b91710896416..92a9f881c50f 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -725,6 +725,9 @@ type Index struct { Columns []cat.IndexColumn + // numSystemColumns is the number of system columns that this index contains. + numSystemColumns int + // IdxZone is the zone associated with the index. This may be inherited from // the parent table, database, or even the default zone. IdxZone *zonepb.ZoneConfig @@ -782,6 +785,11 @@ func (ti *Index) ColumnCount() int { return len(ti.Columns) } +// ColumnCountNoSystemColumns is part of the cat.Index interface. +func (ti *Index) ColumnCountNoSystemColumns() int { + return len(ti.Columns) - ti.numSystemColumns +} + // KeyColumnCount is part of the cat.Index interface. func (ti *Index) KeyColumnCount() int { return ti.KeyCount diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index 99c917c4f748..a7ff2e86a9a2 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -756,7 +756,7 @@ func (c *coster) rowScanCost(tabID opt.TableID, idxOrd int, numScannedCols int) md := c.mem.Metadata() tab := md.Table(tabID) idx := tab.Index(idxOrd) - numCols := idx.ColumnCount() + numCols := idx.ColumnCountNoSystemColumns() // Adjust cost based on how well the current locality matches the index's // zone constraints. diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 6eba6d59a774..93813398e468 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -509,6 +509,12 @@ func (os *optSequence) SequenceMarker() {} type optTable struct { desc *sqlbase.ImmutableTableDescriptor + // systemColumnDescs is the set of implicit system columns for the table. + // It contains column definitions for system columns like the MVCC Timestamp + // column and others. System columns have ordinals larger than the ordinals + // of physical columns and columns in mutations. + systemColumnDescs []cat.Column + // indexes are the inlined wrappers for the table's primary and secondary // indexes. indexes []optIndex @@ -564,6 +570,15 @@ func newOptTable( zone: tblZone, } + // Set up the MVCC timestamp system column. However, we won't add it + // in case a column with the same name already exists in the table. + // Note that the column does not exist when err != nil. This check is done + // for migration purposes. We need to avoid adding the system column if the + // table has a column with this name for some reason. + if _, _, err := desc.FindColumnByName(sqlbase.MVCCTimestampColumnName); err != nil { + ot.systemColumnDescs = append(ot.systemColumnDescs, sqlbase.NewMVCCTimestampColumnDesc()) + } + // Create the table's column mapping from sqlbase.ColumnID to column ordinal. ot.colMap = make(map[sqlbase.ColumnID]int, ot.ColumnCount()) for i, n := 0, ot.ColumnCount(); i < n; i++ { @@ -786,11 +801,14 @@ func (ot *optTable) IsVirtualTable() bool { // ColumnCount is part of the cat.Table interface. func (ot *optTable) ColumnCount() int { - return len(ot.desc.DeletableColumns()) + return len(ot.desc.DeletableColumns()) + len(ot.systemColumnDescs) } // Column is part of the cat.Table interface. func (ot *optTable) Column(i int) cat.Column { + if i >= len(ot.desc.DeletableColumns()) { + return ot.systemColumnDescs[i-len(ot.desc.DeletableColumns())] + } return &ot.desc.DeletableColumns()[i] } @@ -801,8 +819,10 @@ func (ot *optTable) ColumnKind(i int) cat.ColumnKind { return cat.Ordinary case i < len(ot.desc.WritableColumns()): return cat.WriteOnly - default: + case i < len(ot.desc.DeletableColumns()): return cat.DeleteOnly + default: + return cat.SystemColumn } } @@ -904,10 +924,11 @@ type optIndex struct { // otherwise it is desc.StoreColumnIDs. storedCols []sqlbase.ColumnID - indexOrdinal int - numCols int - numKeyCols int - numLaxKeyCols int + indexOrdinal int + numCols int + numKeyCols int + numLaxKeyCols int + numSystemColumns int } var _ cat.Index = &optIndex{} @@ -921,6 +942,7 @@ func (oi *optIndex) init( oi.desc = desc oi.zone = zone oi.indexOrdinal = indexOrdinal + oi.numSystemColumns = 0 if desc == &tab.desc.PrimaryIndex { // Although the primary index contains all columns in the table, the index // descriptor does not contain columns that are not explicitly part of the @@ -937,6 +959,8 @@ func (oi *optIndex) init( } } oi.numCols = tab.ColumnCount() + // The primary index stores some system columns, so remember that here. + oi.numSystemColumns = len(tab.systemColumnDescs) } else { oi.storedCols = desc.StoreColumnIDs oi.numCols = len(desc.ColumnIDs) + len(desc.ExtraColumnIDs) + len(desc.StoreColumnIDs) @@ -946,7 +970,7 @@ func (oi *optIndex) init( notNull := true for _, id := range desc.ColumnIDs { ord, _ := tab.lookupColumnOrdinal(id) - if tab.desc.DeletableColumns()[ord].Nullable { + if tab.Column(ord).IsNullable() { notNull = false break } @@ -998,6 +1022,11 @@ func (oi *optIndex) ColumnCount() int { return oi.numCols } +// ColumnCountNoSystemColumns is part of the cat.Index interface. +func (oi *optIndex) ColumnCountNoSystemColumns() int { + return oi.numCols - oi.numSystemColumns +} + // Predicate is part of the cat.Index interface. It returns the predicate // expression and true if the index is a partial index. If the index is not // partial, the empty string and false is returned. @@ -1679,6 +1708,11 @@ func (oi *optVirtualIndex) ColumnCount() int { return oi.numCols } +// ColumnCountNoSystemColumns is part of the cat.Index interface. +func (oi *optVirtualIndex) ColumnCountNoSystemColumns() int { + return oi.numCols +} + // Predicate is part of the cat.Index interface. func (oi *optVirtualIndex) Predicate() (string, bool) { return "", false diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 0f15cf9064ef..4067ad87aaaf 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -83,6 +83,9 @@ func (ef *execFactory) ConstructScan( scan := ef.planner.Scan() colCfg := makeScanColumnsConfig(table, params.NeededCols) + // Check if any system columns are requested, as they need special handling. + scan.systemColumns, scan.systemColumnOrdinals = collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + sb := span.MakeBuilder(ef.planner.ExecCfg().Codec, tabDesc.TableDesc(), indexDesc) // initTable checks that the current user has the correct privilege to access @@ -520,6 +523,9 @@ func (ef *execFactory) ConstructIndexJoin( tableScan := ef.planner.Scan() + // Check if any system columns are requested, as they need special handling. + tableScan.systemColumns, tableScan.systemColumnOrdinals = collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + if err := tableScan.initTable(context.TODO(), ef.planner, tabDesc, nil, colCfg); err != nil { return nil, err } diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 0c9f990d7bdd..0517f25854cf 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -36,6 +36,10 @@ import ( // this to avoid using log.V in the hot path. const DebugRowFetch = false +// noTimestampColumn is a sentinel value to denote that the MVCC timestamp +// column is not part of the output. +const noTimestampColumn = -1 + type kvBatchFetcher interface { // nextBatch returns the next batch of rows. Returns false in the first // parameter if there are no more keys in the scan. May return either a slice @@ -99,12 +103,13 @@ type tableInfo struct { // The following fields contain MVCC metadata for each row and may be // returned to users of Fetcher immediately after NextRow returns. - // They're not important to ordinary consumers of Fetcher that only - // concern themselves with actual SQL row data. // // rowLastModified is the timestamp of the last time any family in the row // was modified in any way. rowLastModified hlc.Timestamp + // timestampOutputIdx controls at what row ordinal to write the timestamp. + timestampOutputIdx int + // rowIsDeleted is true when the row has been deleted. This is only // meaningful when kv deletion tombstones are returned by the kvBatchFetcher, // which the one used by `StartScan` (the common case) doesnt. Notably, @@ -199,6 +204,11 @@ type Fetcher struct { // when beginning a new scan. traceKV bool + // mvccDecodeStrategy controls whether or not MVCC timestamps should + // be decoded from KV's fetched. It is set if any of the requested tables + // are required to produce an MVCC timestamp system column. + mvccDecodeStrategy MVCCDecodingStrategy + // -- Fields updated during a scan -- kvFetcher *KVFetcher @@ -285,9 +295,10 @@ func (rf *Fetcher) Init( // These slice fields might get re-allocated below, so reslice them from // the old table here in case they've got enough capacity already. - indexColIdx: oldTable.indexColIdx[:0], - keyVals: oldTable.keyVals[:0], - extraVals: oldTable.extraVals[:0], + indexColIdx: oldTable.indexColIdx[:0], + keyVals: oldTable.keyVals[:0], + extraVals: oldTable.extraVals[:0], + timestampOutputIdx: noTimestampColumn, } var err error @@ -327,6 +338,12 @@ func (rf *Fetcher) Init( if tableArgs.ValNeededForCol.Contains(idx) { // The idx-th column is required. table.neededCols.Add(int(col)) + // If this column is the timestamp column, set up the output index. + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(col) + if sysColKind == sqlbase.SystemColumnKind_MVCCTIMESTAMP { + table.timestampOutputIdx = idx + rf.mvccDecodeStrategy = MVCCDecodingRequired + } } } @@ -589,7 +606,7 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, err error) { var ok bool for { - ok, rf.kv, _, err = rf.kvFetcher.NextKV(ctx) + ok, rf.kv, _, err = rf.kvFetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return false, err } @@ -1413,6 +1430,16 @@ func (rf *Fetcher) checkKeyOrdering(ctx context.Context) error { func (rf *Fetcher) finalizeRow() error { table := rf.rowReadyTable + + // If the MVCC timestamp system column was requested, write it to the row. + if table.timestampOutputIdx != noTimestampColumn { + // TODO (rohany): Datums are immutable, so we can't store a DDecimal on the + // fetcher and change its contents with each row. If that assumption gets + // lifted, then we can avoid an allocation of a new decimal datum here. + dec := rf.alloc.NewDDecimal(tree.DDecimal{Decimal: tree.TimestampToDecimal(rf.RowLastModified())}) + table.row[table.timestampOutputIdx] = sqlbase.EncDatum{Datum: dec} + } + // Fill in any missing values with NULLs for i := range table.cols { if rf.valueColsFound == table.neededValueCols { diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 4d067ebd9cf5..194da7e34ca4 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -140,7 +140,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { } row := rowWithMVCCMetadata{ RowIsDeleted: rf.RowIsDeleted(), - RowLastModified: tree.TimestampToDecimal(rf.RowLastModified()).String(), + RowLastModified: tree.TimestampToDecimalDatum(rf.RowLastModified()).String(), } for _, datum := range datums { if datum == tree.DNull { diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 3b2e439030fe..a4da435efb14 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // KVFetcher wraps kvBatchFetcher, providing a NextKV interface that returns the @@ -53,11 +54,22 @@ func newKVFetcher(batchFetcher kvBatchFetcher) *KVFetcher { } } +// MVCCDecodingStrategy controls if and how the fetcher should decode MVCC +// timestamps from returned KV's. +type MVCCDecodingStrategy int + +const ( + // MVCCDecodingNotRequired is used when timestamps aren't needed. + MVCCDecodingNotRequired MVCCDecodingStrategy = iota + // MVCCDecodingRequired is used when timestamps are needed. + MVCCDecodingRequired +) + // NextKV returns the next kv from this fetcher. Returns false if there are no // more kvs to fetch, the kv that was fetched, and any errors that may have // occurred. func (f *KVFetcher) NextKV( - ctx context.Context, + ctx context.Context, mvccDecodeStrategy MVCCDecodingStrategy, ) (ok bool, kv roachpb.KeyValue, newSpan bool, err error) { for { newSpan = f.newSpan @@ -71,14 +83,21 @@ func (f *KVFetcher) NextKV( var key []byte var rawBytes []byte var err error - key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) + var ts hlc.Timestamp + switch mvccDecodeStrategy { + case MVCCDecodingRequired: + key, ts, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValue(f.batchResponse) + case MVCCDecodingNotRequired: + key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) + } if err != nil { return false, kv, false, err } return true, roachpb.KeyValue{ Key: key, Value: roachpb.Value{ - RawBytes: rawBytes, + RawBytes: rawBytes, + Timestamp: ts, }, }, newSpan, nil } diff --git a/pkg/sql/rowexec/indexjoiner.go b/pkg/sql/rowexec/indexjoiner.go index ec2f6c8954a6..31cfcd84f4cf 100644 --- a/pkg/sql/rowexec/indexjoiner.go +++ b/pkg/sql/rowexec/indexjoiner.go @@ -79,10 +79,23 @@ func newIndexJoiner( batchSize: indexJoinerBatchSize, } needMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic + + colIdxMap := ij.desc.ColumnIdxMapWithMutations(needMutations) + resultTypes := ij.desc.ColumnTypesWithMutations(needMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&ij.desc, spec.SystemColumns) + if err != nil { + return nil, err + } + resultTypes = append(resultTypes, sysColTypes...) + for i := range sysColDescs { + colIdxMap[sysColDescs[i].ID] = len(colIdxMap) + } + if err := ij.Init( ij, post, - ij.desc.ColumnTypesWithMutations(needMutations), + resultTypes, flowCtx, processorID, output, @@ -102,14 +115,15 @@ func newIndexJoiner( flowCtx, &fetcher, &ij.desc, - 0, /* primary index */ - ij.desc.ColumnIdxMapWithMutations(needMutations), + 0, /* indexIdx */ + colIdxMap, false, /* reverse */ ij.Out.NeededColumns(), false, /* isCheck */ &ij.alloc, spec.Visibility, spec.LockingStrength, + sysColDescs, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index f6300143a76f..47caecda6da0 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -255,7 +255,7 @@ func newInvertedJoiner( _, _, err = initRowFetcher( flowCtx, &fetcher, &ij.desc, int(spec.IndexIdx), ij.colIdxMap, false, /* reverse */ allIndexCols, false /* isCheck */, &ij.alloc, execinfra.ScanVisibilityPublic, - sqlbase.ScanLockingStrength_FOR_NONE, + sqlbase.ScanLockingStrength_FOR_NONE, nil, /* systemColumns */ ) if err != nil { return nil, err diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index cf8028099c1e..ccd31255a979 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -164,7 +164,7 @@ func newJoinReader( var fetcher row.Fetcher _, _, err = initRowFetcher( flowCtx, &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ - neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, spec.LockingStrength, + neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, spec.LockingStrength, nil, /* systemColumns */ ) if err != nil { return nil, err diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index 7365d2fd43a7..04e8f272655e 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -64,6 +64,7 @@ func initRowFetcher( alloc *sqlbase.DatumAlloc, scanVisibility execinfrapb.ScanVisibility, lockStr sqlbase.ScanLockingStrength, + systemColumns []sqlbase.ColumnDescriptor, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -75,6 +76,9 @@ func initRowFetcher( if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { cols = immutDesc.ReadableColumns } + // Add on any requested system columns. We slice cols to avoid modifying + // the underlying table descriptor. + cols = append(cols[:len(cols):len(cols)], systemColumns...) tableArgs := row.FetcherTableArgs{ Desc: immutDesc, Index: index, diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index a5e365b168a9..f05d48f7bbda 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -124,7 +124,7 @@ func newScrubTableReader( if _, _, err := initRowFetcher( flowCtx, &fetcher, &tr.tableDesc, int(spec.IndexIdx), tr.tableDesc.ColumnIdxMap(), spec.Reverse, neededColumns, true /* isCheck */, &tr.alloc, - execinfra.ScanVisibilityPublic, spec.LockingStrength, + execinfra.ScanVisibilityPublic, spec.LockingStrength, nil, /* systemColumns */ ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 7a78fc418fa8..206de7ebb15b 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -89,12 +89,23 @@ func newTableReader( tr.maxTimestampAge = time.Duration(spec.MaxTimestampAgeNanos) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic - types := spec.Table.ColumnTypesWithMutations(returnMutations) + resultTypes := spec.Table.ColumnTypesWithMutations(returnMutations) + columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&spec.Table, spec.SystemColumns) + if err != nil { + return nil, err + } + resultTypes = append(resultTypes, sysColTypes...) + for i := range sysColDescs { + columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + } + tr.ignoreMisplannedRanges = flowCtx.Local if err := tr.Init( tr, post, - types, + resultTypes, flowCtx, processorID, output, @@ -114,10 +125,19 @@ func newTableReader( neededColumns := tr.Out.NeededColumns() var fetcher row.Fetcher - columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) if _, _, err := initRowFetcher( - flowCtx, &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, spec.Reverse, - neededColumns, spec.IsCheck, &tr.alloc, spec.Visibility, spec.LockingStrength, + flowCtx, + &fetcher, + &spec.Table, + int(spec.IndexIdx), + columnIdxMap, + spec.Reverse, + neededColumns, + spec.IsCheck, + &tr.alloc, + spec.Visibility, + spec.LockingStrength, + sysColDescs, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 8c9508a49d9d..ccd12ac9861b 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -455,6 +455,7 @@ func (z *zigzagJoiner) setupInfo( // NB: zigzag joins are disabled when a row-level locking clause is // supplied, so there is no locking strength on *ZigzagJoinerSpec. sqlbase.ScanLockingStrength_FOR_NONE, + nil, /* systemColumns */ ) if err != nil { return err diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 4beeb7bbb316..89a14bf7d138 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -104,6 +104,12 @@ type scanNode struct { // mode of the Scan. lockingStrength sqlbase.ScanLockingStrength lockingWaitPolicy sqlbase.ScanLockingWaitPolicy + + // systemColumns and systemColumnOrdinals contain information about what + // system columns the scan needs to produce, and what row ordinals to + // write those columns out into. + systemColumns []sqlbase.SystemColumnKind + systemColumnOrdinals []int } // scanColumnsConfig controls the "schema" of a scan node. @@ -271,13 +277,23 @@ func initColsForScan( for _, wc := range colCfg.wantedColumns { var c *sqlbase.ColumnDescriptor var err error - if id := sqlbase.ColumnID(wc); colCfg.visibility == execinfra.ScanVisibilityPublic { - c, err = desc.FindActiveColumnByID(id) + if sqlbase.IsColIDSystemColumn(sqlbase.ColumnID(wc)) { + // If the requested column is a system column, then retrieve the + // corresponding descriptor. + c, err = sqlbase.GetSystemColumnDescriptorFromID(sqlbase.ColumnID(wc)) + if err != nil { + return nil, err + } } else { - c, _, err = desc.FindReadableColumnByID(id) - } - if err != nil { - return cols, err + // Otherwise, collect the descriptors from the table's columns. + if id := sqlbase.ColumnID(wc); colCfg.visibility == execinfra.ScanVisibilityPublic { + c, err = desc.FindActiveColumnByID(id) + } else { + c, _, err = desc.FindReadableColumnByID(id) + } + if err != nil { + return cols, err + } } cols = append(cols, c) diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index dd1b7f6fd44c..019d313bf646 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3224,7 +3224,7 @@ func (ctx *EvalContext) GetClusterTimestamp() *DDecimal { if ts == (hlc.Timestamp{}) { panic(errors.AssertionFailedf("zero cluster timestamp in txn")) } - return TimestampToDecimal(ts) + return TimestampToDecimalDatum(ts) } // HasPlaceholders returns true if this EvalContext's placeholders have been @@ -3236,11 +3236,11 @@ func (ctx *EvalContext) HasPlaceholders() bool { // TimestampToDecimal converts the logical timestamp into a decimal // value with the number of nanoseconds in the integer part and the // logical counter in the decimal part. -func TimestampToDecimal(ts hlc.Timestamp) *DDecimal { +func TimestampToDecimal(ts hlc.Timestamp) apd.Decimal { // Compute Walltime * 10^10 + Logical. // We need 10 decimals for the Logical field because its maximum // value is 4294967295 (2^32-1), a value with 10 decimal digits. - var res DDecimal + var res apd.Decimal val := &res.Coeff val.SetInt64(ts.WallTime) val.Mul(val, big10E10) @@ -3253,8 +3253,17 @@ func TimestampToDecimal(ts hlc.Timestamp) *DDecimal { // Shift 10 decimals to the right, so that the logical // field appears as fractional part. - res.Decimal.Exponent = -10 - return &res + res.Exponent = -10 + return res +} + +// TimestampToDecimalDatum is the same as TimestampToDecimal, but +// returns a datum. +func TimestampToDecimalDatum(ts hlc.Timestamp) *DDecimal { + res := TimestampToDecimal(ts) + return &DDecimal{ + Decimal: res, + } } // TimestampToInexactDTimestamp converts the logical timestamp into an diff --git a/pkg/sql/sqlbase/index_encoding.go b/pkg/sql/sqlbase/index_encoding.go index c39ade84c6f9..9d715a2bc463 100644 --- a/pkg/sql/sqlbase/index_encoding.go +++ b/pkg/sql/sqlbase/index_encoding.go @@ -240,7 +240,7 @@ func MakeSpanFromEncDatums( // retrieve neededCols for the specified table and index. The returned FamilyIDs // are in sorted order. func NeededColumnFamilyIDs( - neededCols util.FastIntSet, table *TableDescriptor, index *IndexDescriptor, + neededColOrdinals util.FastIntSet, table *TableDescriptor, index *IndexDescriptor, ) []FamilyID { if len(table.Families) == 1 { return []FamilyID{table.Families[0].ID} @@ -275,11 +275,14 @@ func NeededColumnFamilyIDs( hasSecondaryEncoding := index.GetEncodingType(table.PrimaryIndex.ID) == SecondaryIndexEncoding // First iterate over the needed columns and look for a few special cases: - // columns which can be decoded from the key and columns whose value is stored - // in family 0. + // * columns which can be decoded from the key and columns whose value is stored + // in family 0. + // * certain system columns, like the MVCC timestamp column require all of the + // column families to be scanned to produce a value. family0Needed := false - nc := neededCols.Copy() - neededCols.ForEach(func(columnOrdinal int) { + mvccColumnRequested := false + nc := neededColOrdinals.Copy() + neededColOrdinals.ForEach(func(columnOrdinal int) { if indexedCols.Contains(columnOrdinal) && !compositeCols.Contains(columnOrdinal) { // We can decode this column from the index key, so no particular family // is needed. @@ -292,8 +295,21 @@ func NeededColumnFamilyIDs( family0Needed = true nc.Remove(columnOrdinal) } + // System column ordinals are larger than the number of columns. + if columnOrdinal >= len(columns) { + mvccColumnRequested = true + } }) + // If the MVCC timestamp column was requested, then bail out. + if mvccColumnRequested { + var families []FamilyID + for i := range table.Families { + families = append(families, table.Families[i].ID) + } + return families + } + // Iterate over the column families to find which ones contain needed columns. // We also keep track of whether all of the needed families' columns are // nullable, since this means we need column family 0 as a sentinel, even if diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index bb92447381cf..8a75d96744d9 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -2002,6 +2002,10 @@ func (desc *TableDescriptor) ValidateTable() error { return pgerror.Newf(pgcode.DuplicateColumn, "duplicate: column %q in the middle of being added, not yet public", column.Name) } + if IsSystemColumnName(column.Name) { + return pgerror.Newf(pgcode.DuplicateColumn, + "column name %q conflicts with a system column name", column.Name) + } columnNames[column.Name] = column.ID if other, ok := columnIDs[column.ID]; ok { diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 1df6b7bb5278..2530de5370c4 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -74,7 +74,48 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0} +} + +// SystemColumnKind is an enum representing the different kind of system +// columns that can be synthesized by the execution engine. +type SystemColumnKind int32 + +const ( + // Default value, unused. + SystemColumnKind_NONE SystemColumnKind = 0 + // A system column containing the value of the MVCC timestamp associated + // with the kv's corresponding to the row. + SystemColumnKind_MVCCTIMESTAMP SystemColumnKind = 1 +) + +var SystemColumnKind_name = map[int32]string{ + 0: "NONE", + 1: "MVCCTIMESTAMP", +} +var SystemColumnKind_value = map[string]int32{ + "NONE": 0, + "MVCCTIMESTAMP": 1, +} + +func (x SystemColumnKind) Enum() *SystemColumnKind { + p := new(SystemColumnKind) + *p = x + return p +} +func (x SystemColumnKind) String() string { + return proto.EnumName(SystemColumnKind_name, int32(x)) +} +func (x *SystemColumnKind) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(SystemColumnKind_value, data, "SystemColumnKind") + if err != nil { + return err + } + *x = SystemColumnKind(value) + return nil +} +func (SystemColumnKind) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_structured_994b169dcbedf9d2, []int{1} } type ForeignKeyReference_Action int32 @@ -119,7 +160,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -159,7 +200,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0, 1} } // The direction of a column in the index. @@ -196,7 +237,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7, 0} } // The type of the index. @@ -233,7 +274,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -276,7 +317,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{8, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{8, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -341,7 +382,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11, 0} } // Direction of mutation. @@ -384,7 +425,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -435,7 +476,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -472,7 +513,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 1} } // Represents the kind of type that this type descriptor represents. @@ -512,7 +553,7 @@ func (x *TypeDescriptor_Kind) UnmarshalJSON(data []byte) error { return nil } func (TypeDescriptor_Kind) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15, 0} } // ForeignKeyReference is deprecated, replaced by ForeignKeyConstraint in v19.2 @@ -542,7 +583,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -590,7 +631,7 @@ func (m *ForeignKeyConstraint) Reset() { *m = ForeignKeyConstraint{} } func (m *ForeignKeyConstraint) String() string { return proto.CompactTextString(m) } func (*ForeignKeyConstraint) ProtoMessage() {} func (*ForeignKeyConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{1} } func (m *ForeignKeyConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -646,13 +687,16 @@ type ColumnDescriptor struct { LogicalColumnID ColumnID `protobuf:"varint,13,opt,name=logical_id,json=logicalId,casttype=ColumnID" json:"logical_id"` // Used to indicate column is used and dropped for ALTER COLUMN TYPE mutation. AlterColumnTypeInProgress bool `protobuf:"varint,14,opt,name=alter_column_type_in_progress,json=alterColumnTypeInProgress" json:"alter_column_type_in_progress"` + // SystemColumnKind represents what kind of system column this column + // descriptor represents, if any. + SystemColumnKind SystemColumnKind `protobuf:"varint,15,opt,name=system_column_kind,json=systemColumnKind,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_column_kind"` } func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{2} + return fileDescriptor_structured_994b169dcbedf9d2, []int{2} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -708,7 +752,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{3} + return fileDescriptor_structured_994b169dcbedf9d2, []int{3} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +798,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{4} + return fileDescriptor_structured_994b169dcbedf9d2, []int{4} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -798,7 +842,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{4, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{4, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -853,7 +897,7 @@ func (m *ShardedDescriptor) Reset() { *m = ShardedDescriptor{} } func (m *ShardedDescriptor) String() string { return proto.CompactTextString(m) } func (*ShardedDescriptor) ProtoMessage() {} func (*ShardedDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{5} + return fileDescriptor_structured_994b169dcbedf9d2, []int{5} } func (m *ShardedDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +942,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -941,7 +985,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -986,7 +1030,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1150,7 +1194,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1201,7 +1245,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{8} + return fileDescriptor_structured_994b169dcbedf9d2, []int{8} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1244,7 +1288,7 @@ func (m *PrimaryKeySwap) Reset() { *m = PrimaryKeySwap{} } func (m *PrimaryKeySwap) String() string { return proto.CompactTextString(m) } func (*PrimaryKeySwap) ProtoMessage() {} func (*PrimaryKeySwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{9} + return fileDescriptor_structured_994b169dcbedf9d2, []int{9} } func (m *PrimaryKeySwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1284,7 +1328,7 @@ func (m *ComputedColumnSwap) Reset() { *m = ComputedColumnSwap{} } func (m *ComputedColumnSwap) String() string { return proto.CompactTextString(m) } func (*ComputedColumnSwap) ProtoMessage() {} func (*ComputedColumnSwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{10} + return fileDescriptor_structured_994b169dcbedf9d2, []int{10} } func (m *ComputedColumnSwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1340,7 +1384,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1648,7 +1692,7 @@ func (m *NameInfo) Reset() { *m = NameInfo{} } func (m *NameInfo) String() string { return proto.CompactTextString(m) } func (*NameInfo) ProtoMessage() {} func (*NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{12} + return fileDescriptor_structured_994b169dcbedf9d2, []int{12} } func (m *NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1825,7 +1869,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2125,7 +2169,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2171,7 +2215,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2211,7 +2255,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 2} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 2} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2248,7 +2292,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 3} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 3} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2291,7 +2335,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 4} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 4} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2331,7 +2375,7 @@ func (m *TableDescriptor_SequenceOpts_SequenceOwner) String() string { } func (*TableDescriptor_SequenceOpts_SequenceOwner) ProtoMessage() {} func (*TableDescriptor_SequenceOpts_SequenceOwner) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 4, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 4, 0} } func (m *TableDescriptor_SequenceOpts_SequenceOwner) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2371,7 +2415,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 5} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 5} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2408,7 +2452,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 6} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 6} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2451,7 +2495,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{14} + return fileDescriptor_structured_994b169dcbedf9d2, []int{14} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2549,7 +2593,7 @@ func (m *TypeDescriptor) Reset() { *m = TypeDescriptor{} } func (m *TypeDescriptor) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor) ProtoMessage() {} func (*TypeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15} } func (m *TypeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2661,7 +2705,7 @@ func (m *TypeDescriptor_EnumMember) Reset() { *m = TypeDescriptor_EnumMe func (m *TypeDescriptor_EnumMember) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor_EnumMember) ProtoMessage() {} func (*TypeDescriptor_EnumMember) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15, 0} } func (m *TypeDescriptor_EnumMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2707,7 +2751,7 @@ func (m *SchemaDescriptor) Reset() { *m = SchemaDescriptor{} } func (m *SchemaDescriptor) String() string { return proto.CompactTextString(m) } func (*SchemaDescriptor) ProtoMessage() {} func (*SchemaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{16} + return fileDescriptor_structured_994b169dcbedf9d2, []int{16} } func (m *SchemaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2796,7 +2840,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{17} + return fileDescriptor_structured_994b169dcbedf9d2, []int{17} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3025,6 +3069,7 @@ func init() { proto.RegisterType((*SchemaDescriptor)(nil), "cockroach.sql.sqlbase.SchemaDescriptor") proto.RegisterType((*Descriptor)(nil), "cockroach.sql.sqlbase.Descriptor") proto.RegisterEnum("cockroach.sql.sqlbase.ConstraintValidity", ConstraintValidity_name, ConstraintValidity_value) + proto.RegisterEnum("cockroach.sql.sqlbase.SystemColumnKind", SystemColumnKind_name, SystemColumnKind_value) proto.RegisterEnum("cockroach.sql.sqlbase.ForeignKeyReference_Action", ForeignKeyReference_Action_name, ForeignKeyReference_Action_value) proto.RegisterEnum("cockroach.sql.sqlbase.ForeignKeyReference_Match", ForeignKeyReference_Match_name, ForeignKeyReference_Match_value) proto.RegisterEnum("cockroach.sql.sqlbase.IndexDescriptor_Direction", IndexDescriptor_Direction_name, IndexDescriptor_Direction_value) @@ -3213,6 +3258,9 @@ func (this *ColumnDescriptor) Equal(that interface{}) bool { if this.AlterColumnTypeInProgress != that1.AlterColumnTypeInProgress { return false } + if this.SystemColumnKind != that1.SystemColumnKind { + return false + } return true } func (this *ColumnFamilyDescriptor) Equal(that interface{}) bool { @@ -4837,6 +4885,9 @@ func (m *ColumnDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x78 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.SystemColumnKind)) return i, nil } @@ -6555,6 +6606,7 @@ func (m *ColumnDescriptor) Size() (n int) { } n += 1 + sovStructured(uint64(m.LogicalColumnID)) n += 2 + n += 1 + sovStructured(uint64(m.SystemColumnKind)) return n } @@ -8260,6 +8312,25 @@ func (m *ColumnDescriptor) Unmarshal(dAtA []byte) error { } } m.AlterColumnTypeInProgress = bool(v != 0) + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumnKind", wireType) + } + m.SystemColumnKind = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SystemColumnKind |= (SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) @@ -14324,273 +14395,276 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_455e90a118345fa6) -} - -var fileDescriptor_structured_455e90a118345fa6 = []byte{ - // 4215 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3b, 0x49, 0x73, 0x1b, 0x57, - 0x7a, 0x68, 0xec, 0xf8, 0xb0, 0xb0, 0xf9, 0x44, 0x51, 0x10, 0xad, 0x21, 0x29, 0xc8, 0xb2, 0x39, - 0x63, 0x9b, 0x94, 0xa9, 0x49, 0x46, 0x63, 0x27, 0x53, 0x06, 0x01, 0x50, 0x04, 0x49, 0x01, 0x74, - 0x93, 0xb2, 0x66, 0xb2, 0x75, 0x9a, 0xe8, 0x07, 0xb0, 0xad, 0x46, 0x37, 0xd4, 0xdd, 0x90, 0xc8, - 0xaa, 0x9c, 0x72, 0x9a, 0x53, 0x2a, 0x97, 0xe4, 0x96, 0x2a, 0x57, 0xe2, 0xaa, 0xcc, 0x29, 0xa9, - 0x5c, 0x92, 0x5b, 0xaa, 0x72, 0x73, 0x4e, 0x99, 0xdc, 0xe6, 0xc4, 0x4a, 0xe8, 0x4b, 0x7e, 0x41, - 0x52, 0xe5, 0x5c, 0x52, 0x6f, 0xeb, 0x05, 0x0b, 0x0d, 0x92, 0xca, 0x45, 0xc5, 0xfe, 0xb6, 0xf7, - 0xde, 0xf7, 0xbe, 0xfd, 0x41, 0x70, 0xcf, 0x7d, 0x65, 0x6e, 0xb8, 0xaf, 0xcc, 0x63, 0xcd, 0xc5, - 0x1b, 0xae, 0xe7, 0x0c, 0x3b, 0xde, 0xd0, 0xc1, 0xfa, 0xfa, 0xc0, 0xb1, 0x3d, 0x1b, 0xdd, 0xee, - 0xd8, 0x9d, 0x97, 0x8e, 0xad, 0x75, 0x4e, 0xd6, 0xdd, 0x57, 0xe6, 0x3a, 0xa7, 0x5b, 0x2a, 0x0f, - 0x3d, 0xc3, 0xdc, 0x38, 0x31, 0x3b, 0x1b, 0x9e, 0xd1, 0xc7, 0xae, 0xa7, 0xf5, 0x07, 0x8c, 0x61, - 0xe9, 0x9d, 0xb0, 0xb8, 0x81, 0x63, 0xbc, 0x36, 0x4c, 0xdc, 0xc3, 0x1c, 0x79, 0x9b, 0x20, 0xbd, - 0xb3, 0x01, 0x76, 0xd9, 0xbf, 0x1c, 0x7c, 0xb7, 0x87, 0xed, 0x8d, 0x1e, 0xb6, 0x0d, 0x4b, 0xc7, - 0xa7, 0x1b, 0x1d, 0xdb, 0xea, 0x1a, 0x3d, 0x8e, 0x5a, 0xe8, 0xd9, 0x3d, 0x9b, 0xfe, 0xb9, 0x41, - 0xfe, 0x62, 0xd0, 0xca, 0x9f, 0xa6, 0xe0, 0xd6, 0xb6, 0xed, 0x60, 0xa3, 0x67, 0xed, 0xe1, 0x33, - 0x05, 0x77, 0xb1, 0x83, 0xad, 0x0e, 0x46, 0xab, 0x90, 0xf2, 0xb4, 0x63, 0x13, 0x97, 0xa5, 0x55, - 0x69, 0xad, 0xb8, 0x05, 0xdf, 0x9c, 0xaf, 0xc4, 0xbe, 0x3b, 0x5f, 0x89, 0x37, 0xeb, 0x0a, 0x43, - 0xa0, 0x87, 0x90, 0xa2, 0xab, 0x94, 0xe3, 0x94, 0x62, 0x8e, 0x53, 0x64, 0x9a, 0x04, 0x48, 0xc8, - 0x28, 0x16, 0x95, 0x21, 0x69, 0x69, 0x7d, 0x5c, 0x4e, 0xac, 0x4a, 0x6b, 0xb9, 0xad, 0x24, 0xa1, - 0x52, 0x28, 0x04, 0xed, 0x41, 0xf6, 0xb5, 0x66, 0x1a, 0xba, 0xe1, 0x9d, 0x95, 0x93, 0xab, 0xd2, - 0x5a, 0x69, 0xf3, 0x87, 0xeb, 0x13, 0x75, 0xb4, 0x5e, 0xb3, 0x2d, 0xd7, 0x73, 0x34, 0xc3, 0xf2, - 0xbe, 0xe0, 0x0c, 0x5c, 0x90, 0x2f, 0x00, 0x3d, 0x82, 0x79, 0xf7, 0x44, 0x73, 0xb0, 0xae, 0x0e, - 0x1c, 0xdc, 0x35, 0x4e, 0x55, 0x13, 0x5b, 0xe5, 0xd4, 0xaa, 0xb4, 0x96, 0xe2, 0xa4, 0x73, 0x0c, - 0x7d, 0x40, 0xb1, 0xfb, 0xd8, 0x42, 0x47, 0x90, 0xb3, 0x2d, 0x55, 0xc7, 0x26, 0xf6, 0x70, 0x39, - 0x4d, 0xd7, 0xff, 0x78, 0xca, 0xfa, 0x13, 0x14, 0xb4, 0x5e, 0xed, 0x78, 0x86, 0x6d, 0x89, 0x7d, - 0xd8, 0x56, 0x9d, 0x0a, 0xe2, 0x52, 0x87, 0x03, 0x5d, 0xf3, 0x70, 0x39, 0x73, 0x63, 0xa9, 0xcf, - 0xa9, 0x20, 0xb4, 0x0f, 0xa9, 0xbe, 0xe6, 0x75, 0x4e, 0xca, 0x59, 0x2a, 0xf1, 0xd1, 0x15, 0x24, - 0x3e, 0x23, 0x7c, 0x5c, 0x20, 0x13, 0x52, 0x79, 0x01, 0x69, 0xb6, 0x0e, 0x2a, 0x42, 0xae, 0xd5, - 0x56, 0xab, 0xb5, 0xa3, 0x66, 0xbb, 0x25, 0xc7, 0x50, 0x01, 0xb2, 0x4a, 0xe3, 0xf0, 0x48, 0x69, - 0xd6, 0x8e, 0x64, 0x89, 0x7c, 0x1d, 0x36, 0x8e, 0xd4, 0xd6, 0xf3, 0xfd, 0x7d, 0x39, 0x8e, 0xe6, - 0x20, 0x4f, 0xbe, 0xea, 0x8d, 0xed, 0xea, 0xf3, 0xfd, 0x23, 0x39, 0x81, 0xf2, 0x90, 0xa9, 0x55, - 0x0f, 0x6b, 0xd5, 0x7a, 0x43, 0x4e, 0x2e, 0x25, 0x7f, 0xf5, 0xf5, 0x72, 0xac, 0xf2, 0x08, 0x52, - 0x74, 0x39, 0x04, 0x90, 0x3e, 0x6c, 0x3e, 0x3b, 0xd8, 0x6f, 0xc8, 0x31, 0x94, 0x85, 0xe4, 0x36, - 0x11, 0x21, 0x11, 0x8e, 0x83, 0xaa, 0x72, 0xd4, 0xac, 0xee, 0xcb, 0x71, 0xc6, 0xf1, 0x49, 0xf2, - 0xbf, 0xbe, 0x5a, 0x91, 0x2a, 0xff, 0x9e, 0x82, 0x85, 0x60, 0xef, 0xc1, 0x6d, 0xa3, 0x1a, 0xcc, - 0xd9, 0x8e, 0xd1, 0x33, 0x2c, 0x95, 0xda, 0x9c, 0x6a, 0xe8, 0xdc, 0x1e, 0xdf, 0x21, 0xe7, 0xb9, - 0x38, 0x5f, 0x29, 0xb6, 0x29, 0xfa, 0x88, 0x60, 0x9b, 0x75, 0x6e, 0xa0, 0x45, 0x3b, 0x04, 0xd4, - 0xd1, 0x1e, 0xcc, 0x73, 0x21, 0x1d, 0xdb, 0x1c, 0xf6, 0x2d, 0xd5, 0xd0, 0xdd, 0x72, 0x7c, 0x35, - 0xb1, 0x56, 0xdc, 0x5a, 0xb9, 0x38, 0x5f, 0x99, 0x63, 0x22, 0x6a, 0x14, 0xd7, 0xac, 0xbb, 0xdf, - 0x9d, 0xaf, 0x64, 0xc5, 0x87, 0xc2, 0x97, 0xe7, 0xdf, 0xba, 0x8b, 0x5e, 0xc0, 0x6d, 0x47, 0xe8, - 0x56, 0x0f, 0x0b, 0x4c, 0x50, 0x81, 0x0f, 0x2e, 0xce, 0x57, 0x6e, 0xf9, 0xca, 0xd7, 0x27, 0x0b, - 0xbd, 0xe5, 0x8c, 0x12, 0xe8, 0x2e, 0x6a, 0x43, 0x08, 0x1c, 0x1c, 0x37, 0x49, 0x8f, 0xbb, 0xc2, - 0x8f, 0x3b, 0x1f, 0x88, 0x8e, 0x1e, 0x79, 0xde, 0x19, 0x41, 0xe8, 0xbe, 0xe3, 0xa5, 0x2e, 0x75, - 0xbc, 0xf4, 0x4d, 0x1d, 0x2f, 0xe2, 0x46, 0x99, 0xff, 0x17, 0x37, 0xca, 0xbe, 0x75, 0x37, 0xca, - 0xbd, 0x05, 0x37, 0x62, 0xb6, 0xbb, 0x9b, 0xcc, 0x82, 0x9c, 0xdf, 0x4d, 0x66, 0xf3, 0x72, 0x61, - 0x37, 0x99, 0x2d, 0xc8, 0xc5, 0xdd, 0x64, 0xb6, 0x28, 0x97, 0x2a, 0x7f, 0x99, 0x04, 0x99, 0xdd, - 0x6e, 0x1d, 0xbb, 0x1d, 0xc7, 0x18, 0x78, 0xb6, 0xe3, 0xdf, 0x89, 0x34, 0x76, 0x27, 0xef, 0x41, - 0xdc, 0xd0, 0x79, 0x28, 0x5d, 0xe4, 0xb7, 0x1d, 0xa7, 0xd7, 0x1b, 0xd8, 0x4d, 0xdc, 0xd0, 0xd1, - 0x3a, 0x24, 0x49, 0xbc, 0xa7, 0xe1, 0x34, 0xbf, 0xb9, 0x34, 0x7a, 0x02, 0xdc, 0x5f, 0x67, 0xe9, - 0xe0, 0x48, 0xa1, 0x74, 0x68, 0x15, 0xb2, 0xd6, 0xd0, 0x34, 0x69, 0x28, 0x27, 0xb6, 0x94, 0x15, - 0x4a, 0x11, 0x50, 0x74, 0x1f, 0x0a, 0x3a, 0xee, 0x6a, 0x43, 0xd3, 0x53, 0xf1, 0xe9, 0xc0, 0x61, - 0xf6, 0xa2, 0xe4, 0x39, 0xac, 0x71, 0x3a, 0x70, 0xd0, 0x3d, 0x48, 0x9f, 0x18, 0xba, 0x8e, 0x2d, - 0x6a, 0x2e, 0x42, 0x04, 0x87, 0xa1, 0x4d, 0x98, 0x1f, 0xba, 0xd8, 0x55, 0x5d, 0xfc, 0x6a, 0x48, - 0x74, 0x45, 0xdd, 0x01, 0xa8, 0x3b, 0xa4, 0xb9, 0x79, 0xce, 0x11, 0x82, 0x43, 0x8e, 0x27, 0xd6, - 0x7e, 0x1f, 0x0a, 0x1d, 0xbb, 0x3f, 0x18, 0x7a, 0x98, 0x2d, 0x9a, 0x67, 0x8b, 0x72, 0x18, 0x5d, - 0x74, 0x13, 0xe6, 0xed, 0x37, 0xd6, 0x88, 0xd8, 0x42, 0x54, 0x2c, 0x21, 0x08, 0x8b, 0xdd, 0x02, - 0x30, 0xed, 0x9e, 0xd1, 0xd1, 0x4c, 0xe2, 0x3b, 0x45, 0xaa, 0xcd, 0x07, 0x5c, 0x9b, 0x73, 0xfb, - 0x0c, 0x23, 0xd4, 0x19, 0x51, 0x6d, 0x8e, 0xb3, 0x35, 0x75, 0xb4, 0x0d, 0x3f, 0xd0, 0x4c, 0x0f, - 0x3b, 0xc2, 0xb9, 0x89, 0x1a, 0x55, 0xc3, 0x52, 0x07, 0x8e, 0xdd, 0x73, 0xb0, 0xeb, 0x96, 0x4b, - 0x21, 0x1d, 0xdc, 0xa5, 0xa4, 0x4c, 0xcc, 0xd1, 0xd9, 0x00, 0x37, 0xad, 0x03, 0x4e, 0xe6, 0x9b, - 0x47, 0x56, 0xce, 0xed, 0x26, 0xb3, 0x39, 0x19, 0x76, 0x93, 0xd9, 0x8c, 0x9c, 0xad, 0xfc, 0x59, - 0x1c, 0x16, 0x19, 0xc3, 0xb6, 0xd6, 0x37, 0xcc, 0xb3, 0x9b, 0x9a, 0x07, 0x93, 0xc2, 0xcd, 0x83, - 0xea, 0x95, 0x6e, 0x9b, 0xb0, 0xb1, 0xa8, 0x44, 0xf5, 0x4a, 0x60, 0x2d, 0x02, 0x42, 0x4f, 0x00, - 0x42, 0x61, 0x2b, 0x49, 0x15, 0x7a, 0xf7, 0xe2, 0x7c, 0x25, 0x37, 0x39, 0x58, 0xe5, 0x3a, 0xa1, - 0x10, 0x35, 0x2f, 0x2c, 0xc5, 0x97, 0x40, 0xcd, 0x25, 0xa4, 0xe4, 0x3a, 0x23, 0x98, 0xa8, 0xe4, - 0x39, 0x3d, 0x82, 0xd4, 0x79, 0xf4, 0xff, 0xa7, 0x38, 0x2c, 0x34, 0x2d, 0x0f, 0x3b, 0x26, 0xd6, - 0x5e, 0xe3, 0x90, 0x3a, 0x7e, 0x0e, 0x39, 0xcd, 0xea, 0x60, 0xd7, 0xb3, 0x1d, 0xb7, 0x2c, 0xad, - 0x26, 0xd6, 0xf2, 0x9b, 0x3f, 0x9e, 0xe2, 0xb2, 0x93, 0xf8, 0xd7, 0xab, 0x9c, 0x99, 0x6b, 0x32, - 0x10, 0xb6, 0xf4, 0xcf, 0x12, 0x64, 0x05, 0x16, 0x3d, 0x82, 0xec, 0x48, 0x76, 0xb9, 0xcd, 0x4f, - 0x93, 0x89, 0x06, 0xd9, 0x8c, 0xc7, 0x43, 0xeb, 0x6f, 0x41, 0x96, 0x16, 0x37, 0xaa, 0x7f, 0x27, - 0x4b, 0x82, 0x83, 0x57, 0x3f, 0xe1, 0x42, 0x28, 0x43, 0x69, 0x9b, 0x3a, 0xaa, 0x4d, 0xaa, 0x51, - 0x12, 0x94, 0xff, 0x8e, 0xd0, 0xdf, 0x61, 0xb4, 0x4a, 0x19, 0x2b, 0x5b, 0x98, 0xce, 0xb8, 0xe6, - 0xfe, 0x51, 0x82, 0x79, 0xc2, 0xa0, 0x63, 0x3d, 0xa4, 0xb6, 0x07, 0x00, 0x86, 0xab, 0xba, 0x0c, - 0x4e, 0x4f, 0x24, 0xac, 0x35, 0x67, 0xb8, 0x9c, 0xdc, 0x37, 0xb5, 0xf8, 0x98, 0xa9, 0xfd, 0x14, - 0x8a, 0x94, 0x57, 0x3d, 0x1e, 0x76, 0x5e, 0x62, 0xcf, 0xa5, 0x3b, 0x4c, 0x6d, 0x2d, 0xf0, 0x1d, - 0x16, 0xa8, 0x84, 0x2d, 0x86, 0x53, 0x0a, 0x6e, 0xe8, 0x6b, 0xcc, 0xfa, 0x92, 0x63, 0xd6, 0xc7, - 0x37, 0xfe, 0x3f, 0x09, 0x58, 0x3c, 0xd0, 0x1c, 0xcf, 0x20, 0x61, 0xda, 0xb0, 0x7a, 0xa1, 0xdd, - 0x3f, 0x84, 0xbc, 0x35, 0xec, 0x73, 0x03, 0x73, 0xf9, 0x85, 0xb0, 0xfd, 0x81, 0x35, 0xec, 0x33, - 0xdb, 0x71, 0xd1, 0x3e, 0x24, 0x4d, 0xc3, 0xf5, 0x68, 0x1e, 0xcf, 0x6f, 0x6e, 0x4e, 0x31, 0x8b, - 0xc9, 0x6b, 0xac, 0xef, 0x1b, 0xae, 0x27, 0xce, 0x4c, 0xa4, 0xa0, 0x36, 0xa4, 0x1c, 0xcd, 0xea, - 0x61, 0xea, 0x2f, 0xf9, 0xcd, 0xc7, 0x57, 0x13, 0xa7, 0x10, 0x56, 0x91, 0x1b, 0xa8, 0x9c, 0xa5, - 0xbf, 0x92, 0x20, 0x49, 0x56, 0xb9, 0xc4, 0xa5, 0x17, 0x21, 0xfd, 0x5a, 0x33, 0x87, 0x98, 0xd5, - 0x22, 0x05, 0x85, 0x7f, 0xa1, 0x3f, 0x84, 0x39, 0x77, 0x78, 0x3c, 0x08, 0x2d, 0xc5, 0x83, 0xfd, - 0x47, 0x57, 0xda, 0x95, 0x5f, 0xf6, 0x46, 0x65, 0xb1, 0x0b, 0x58, 0x7a, 0x05, 0x29, 0xba, 0xeb, - 0x4b, 0xf6, 0x77, 0x1f, 0x0a, 0x9e, 0xad, 0xe2, 0xd3, 0x8e, 0x39, 0x74, 0x8d, 0xd7, 0xcc, 0x52, - 0x0a, 0x4a, 0xde, 0xb3, 0x1b, 0x02, 0x84, 0x1e, 0x42, 0xa9, 0xeb, 0xd8, 0x7d, 0xd5, 0xb0, 0x04, - 0x51, 0x82, 0x12, 0x15, 0x09, 0xb4, 0x29, 0x80, 0x11, 0x93, 0xfd, 0x8b, 0x02, 0xcc, 0x51, 0xc7, - 0x98, 0x29, 0xec, 0x3d, 0x0c, 0x85, 0xbd, 0xdb, 0x91, 0xb0, 0xe7, 0x7b, 0x17, 0x89, 0x7a, 0xf7, - 0x20, 0x3d, 0xb4, 0x8c, 0x57, 0x43, 0xb6, 0xbe, 0x9f, 0x9f, 0x18, 0x6c, 0x06, 0xab, 0x44, 0x1f, - 0x02, 0x22, 0xa1, 0x00, 0xab, 0x11, 0xc2, 0x14, 0x25, 0x94, 0x29, 0xa6, 0x36, 0x35, 0x82, 0xa6, - 0xaf, 0x10, 0x41, 0x77, 0x40, 0xc6, 0xa7, 0x9e, 0xa3, 0x85, 0x0b, 0xc7, 0x0c, 0xe5, 0x5f, 0xbe, - 0x38, 0x5f, 0x29, 0x35, 0x08, 0x6e, 0xb2, 0x90, 0x12, 0x0e, 0xe1, 0x74, 0x62, 0x25, 0xf3, 0x5c, - 0x86, 0x6e, 0x38, 0x98, 0x96, 0x3b, 0x6e, 0x39, 0xbb, 0x9a, 0xb8, 0xa4, 0xac, 0x19, 0x51, 0xfb, - 0x7a, 0x5d, 0x30, 0x2a, 0x32, 0x13, 0xe5, 0x03, 0x5c, 0x74, 0x08, 0xf9, 0x2e, 0xab, 0x82, 0xd4, - 0x97, 0xf8, 0x8c, 0xd6, 0x4b, 0xf9, 0xcd, 0x1f, 0xcd, 0x5e, 0x2f, 0x6d, 0xa5, 0xc9, 0x15, 0x94, - 0x25, 0x05, 0xba, 0x3e, 0x12, 0xbd, 0x80, 0x62, 0xa8, 0xc4, 0x3d, 0x3e, 0xa3, 0x45, 0xc2, 0xf5, - 0xc4, 0x16, 0x02, 0x41, 0x5b, 0x67, 0xe8, 0x73, 0x00, 0xc3, 0x4f, 0x00, 0xb4, 0x96, 0xc8, 0x6f, - 0x7e, 0x70, 0x85, 0x4c, 0x21, 0xe2, 0x4b, 0x20, 0x04, 0xbd, 0x80, 0x52, 0xf0, 0x45, 0x37, 0x5b, - 0xb8, 0xf2, 0x66, 0x99, 0xd4, 0x62, 0x48, 0xce, 0x16, 0xa9, 0x97, 0x17, 0x48, 0x95, 0x63, 0xbb, - 0x86, 0x87, 0xc3, 0x66, 0x50, 0xa4, 0x66, 0x50, 0xb9, 0x38, 0x5f, 0x41, 0x35, 0x81, 0x9f, 0x6c, - 0x0a, 0xa8, 0x33, 0x82, 0x67, 0x86, 0x15, 0x31, 0x60, 0x22, 0xb1, 0x14, 0x18, 0xd6, 0x61, 0x60, - 0xc2, 0x63, 0x86, 0x15, 0x32, 0x6f, 0xd6, 0xe0, 0x14, 0x22, 0xb1, 0x67, 0xee, 0xfa, 0xb1, 0x27, - 0x22, 0x08, 0x35, 0x78, 0xe5, 0x2a, 0xd3, 0xda, 0xfb, 0x83, 0x19, 0x8d, 0x94, 0x14, 0x55, 0x22, - 0x24, 0xd0, 0x82, 0xf6, 0x31, 0xa0, 0x8e, 0x83, 0x35, 0x0f, 0xeb, 0xa4, 0x72, 0x34, 0x8d, 0x8e, - 0xe1, 0x99, 0x67, 0xe5, 0xf9, 0x90, 0xdf, 0xcf, 0x73, 0x7c, 0xc3, 0x47, 0xa3, 0x27, 0x90, 0x79, - 0x8d, 0x1d, 0xd7, 0xb0, 0xad, 0x32, 0xa2, 0xc1, 0x64, 0x99, 0x4f, 0x2b, 0x16, 0x47, 0xd6, 0xfb, - 0x82, 0x51, 0x29, 0x82, 0x1c, 0xed, 0x40, 0x11, 0x5b, 0x1d, 0x5b, 0x37, 0xac, 0x1e, 0xad, 0x04, - 0xcb, 0xb7, 0x82, 0x7a, 0xe7, 0xbb, 0xf3, 0x95, 0x77, 0x46, 0xf8, 0x1b, 0x9c, 0x96, 0x6c, 0x5b, - 0x29, 0xe0, 0xd0, 0x17, 0xda, 0x81, 0x8c, 0xc8, 0xc9, 0x0b, 0x54, 0xa7, 0x6b, 0x53, 0x54, 0x30, - 0x96, 0xd1, 0xf9, 0xb9, 0x04, 0x3b, 0xa9, 0xe9, 0x75, 0xc3, 0x25, 0xb5, 0x88, 0x5e, 0xbe, 0x1d, - 0xae, 0xe9, 0x05, 0x14, 0xd5, 0x00, 0x7a, 0xd8, 0x56, 0xd9, 0xfc, 0xa7, 0xbc, 0x48, 0x97, 0x5b, - 0x0e, 0x2d, 0xd7, 0xc3, 0xf6, 0xba, 0x98, 0x12, 0x91, 0x26, 0xaf, 0x6b, 0xf4, 0x44, 0x89, 0xd0, - 0xc3, 0x36, 0x03, 0xa0, 0x0a, 0xe4, 0x06, 0x0e, 0xd6, 0x8d, 0x0e, 0xe9, 0xc1, 0xee, 0x84, 0x62, - 0x73, 0x00, 0xae, 0x2c, 0x43, 0xce, 0x8f, 0x1a, 0x28, 0x03, 0x89, 0xea, 0x61, 0x8d, 0xb5, 0xfc, - 0xf5, 0xc6, 0x61, 0x4d, 0x96, 0x2a, 0xf7, 0x21, 0x49, 0x0f, 0x9f, 0x87, 0xcc, 0x76, 0x5b, 0x79, - 0x51, 0x55, 0xea, 0x6c, 0xcc, 0xd0, 0x6c, 0x7d, 0xd1, 0x50, 0x8e, 0x1a, 0x75, 0x59, 0xe4, 0x85, - 0x7f, 0x49, 0x00, 0x0a, 0xba, 0xcd, 0x23, 0x9b, 0x77, 0x6c, 0x3d, 0x98, 0xeb, 0xf8, 0x50, 0x76, - 0x01, 0xd2, 0x6a, 0x7c, 0xad, 0xb4, 0xf9, 0xe4, 0x7b, 0x3b, 0x56, 0x21, 0x23, 0x0c, 0x0a, 0x8c, - 0xa9, 0xd4, 0x89, 0x40, 0x43, 0xf5, 0x50, 0x7c, 0x24, 0x07, 0x29, 0x90, 0xea, 0x9c, 0xe0, 0xce, - 0x4b, 0x9e, 0x85, 0x7f, 0x7b, 0xca, 0xc2, 0xb4, 0x54, 0x0c, 0x19, 0x6e, 0x8d, 0xf0, 0x04, 0x4b, - 0x8b, 0xf2, 0x80, 0x8a, 0x42, 0x4a, 0x34, 0xbc, 0x26, 0x2f, 0x8d, 0x58, 0x93, 0x26, 0x23, 0x22, - 0x62, 0x85, 0xa2, 0xeb, 0x13, 0x98, 0xb3, 0x6c, 0x4f, 0x25, 0x7d, 0x1d, 0x8f, 0x02, 0xb4, 0x5b, - 0x2b, 0x6e, 0xc9, 0xdc, 0x56, 0x03, 0x9f, 0x2f, 0x5a, 0xb6, 0xd7, 0x1a, 0x9a, 0xbc, 0x15, 0xaa, - 0x7c, 0x02, 0xa5, 0xa8, 0x8e, 0x50, 0x0e, 0x52, 0xb5, 0x9d, 0x46, 0x6d, 0x4f, 0x8e, 0xa1, 0x39, - 0xc8, 0x6f, 0xb7, 0x95, 0x46, 0xf3, 0x69, 0x4b, 0xdd, 0x6b, 0xfc, 0x82, 0x8d, 0x85, 0x5a, 0x6d, - 0x31, 0x16, 0xf2, 0xbb, 0x9c, 0x94, 0x9c, 0xae, 0xfc, 0xb7, 0x04, 0xa5, 0x03, 0xc7, 0xe8, 0x6b, - 0xce, 0xd9, 0x1e, 0x3e, 0x3b, 0x7c, 0xa3, 0x0d, 0xd0, 0x67, 0xb0, 0x60, 0xe1, 0x37, 0xea, 0x80, - 0x41, 0x55, 0xbf, 0x6a, 0x96, 0x26, 0xcf, 0x0c, 0xe7, 0x2d, 0xfc, 0x86, 0x4b, 0x68, 0xf2, 0xa2, - 0xf9, 0x43, 0xc8, 0xdb, 0xa6, 0xce, 0x38, 0xb1, 0x98, 0xdb, 0xe4, 0xc3, 0x4c, 0x60, 0x9b, 0x7a, - 0x93, 0xa1, 0x09, 0x35, 0x59, 0x4f, 0x50, 0x27, 0x26, 0x50, 0x5b, 0xf8, 0x8d, 0xa0, 0xfe, 0x0c, - 0x16, 0x88, 0xec, 0xb1, 0xdd, 0x25, 0xa7, 0xec, 0xce, 0x36, 0xf5, 0xe8, 0xee, 0xb8, 0xf1, 0xfe, - 0xbd, 0x04, 0x34, 0x68, 0x0f, 0x3d, 0x31, 0xd1, 0xa1, 0x87, 0xff, 0x31, 0x14, 0xc9, 0x66, 0x82, - 0x5e, 0x49, 0x9a, 0x72, 0x1f, 0x64, 0xcf, 0x22, 0x02, 0x13, 0x2e, 0xb2, 0xa9, 0x80, 0x2b, 0x3e, - 0x8d, 0xcb, 0x36, 0xfd, 0xf9, 0x11, 0x7a, 0x1f, 0x0a, 0x86, 0x45, 0x82, 0x16, 0x6f, 0xa8, 0xc3, - 0xe3, 0xd6, 0x3c, 0xc7, 0x90, 0xb6, 0x9a, 0xef, 0xf8, 0x6f, 0xd2, 0x80, 0x02, 0x63, 0x7d, 0x36, - 0xf4, 0x34, 0xea, 0xc1, 0x55, 0x48, 0x73, 0xd3, 0x91, 0xa8, 0x49, 0xbe, 0x3f, 0xd5, 0xcb, 0xa2, - 0x83, 0x8d, 0x9d, 0x98, 0xc2, 0x19, 0xd1, 0xcf, 0xc2, 0x63, 0xe1, 0xfc, 0xe6, 0x7b, 0xb3, 0xc5, - 0xf9, 0x9d, 0x98, 0x98, 0x17, 0xef, 0x41, 0xca, 0xf5, 0x48, 0xc4, 0x49, 0xd0, 0x3c, 0xb1, 0x31, - 0x85, 0x7f, 0x7c, 0xf3, 0xeb, 0x87, 0x84, 0x4d, 0xf8, 0x19, 0x95, 0x81, 0x5e, 0x40, 0xce, 0x2f, - 0x8f, 0xf8, 0x8c, 0xf9, 0xf1, 0xec, 0x02, 0xfd, 0xc8, 0x26, 0xe2, 0x9e, 0x2f, 0x0b, 0x55, 0x21, - 0xdf, 0xe7, 0x64, 0x41, 0x13, 0xbc, 0xca, 0x2b, 0x54, 0x10, 0x12, 0x68, 0xa5, 0x1a, 0xfa, 0x52, - 0x40, 0x30, 0x35, 0x69, 0x14, 0x77, 0x6c, 0xd3, 0x3c, 0xd6, 0x3a, 0x2f, 0xe9, 0xdc, 0xcc, 0x8f, - 0xe2, 0x02, 0x8a, 0xf6, 0x48, 0x9d, 0x29, 0xfc, 0x92, 0x4e, 0xc1, 0xf2, 0x33, 0x4c, 0xea, 0x44, - 0xdc, 0xdb, 0x89, 0x29, 0x21, 0x76, 0xd4, 0x86, 0xd2, 0x20, 0xe2, 0x9b, 0xbc, 0xa8, 0x7b, 0x38, - 0x2d, 0xb3, 0x47, 0x88, 0x77, 0x62, 0xca, 0x08, 0x3b, 0xfa, 0x7d, 0x40, 0x9d, 0x31, 0x9b, 0x2f, - 0xc3, 0xf7, 0xec, 0x72, 0x94, 0x61, 0x27, 0xa6, 0x4c, 0x10, 0x53, 0xf9, 0x0c, 0x52, 0xf4, 0x3a, - 0x49, 0xe2, 0x78, 0xde, 0xda, 0x6b, 0xb5, 0x5f, 0xb4, 0x58, 0x2c, 0xaa, 0x37, 0xf6, 0x1b, 0x47, - 0x0d, 0xb5, 0xdd, 0xda, 0x27, 0xb1, 0xe8, 0x2e, 0xdc, 0xe6, 0x80, 0x6a, 0xab, 0xae, 0xbe, 0x50, - 0x9a, 0x02, 0x15, 0xaf, 0xac, 0x85, 0x33, 0x53, 0x16, 0x92, 0xad, 0x76, 0xab, 0x21, 0xc7, 0x68, - 0x8e, 0xaa, 0xd7, 0x65, 0x89, 0xe6, 0x28, 0xa5, 0x7d, 0x20, 0x42, 0xd8, 0x56, 0x01, 0x40, 0xf7, - 0x4d, 0x60, 0x37, 0x99, 0x4d, 0xcb, 0x99, 0xca, 0xdf, 0x4a, 0x90, 0x25, 0x25, 0x7f, 0xd3, 0xea, - 0xda, 0xe8, 0x31, 0xe4, 0x06, 0x9a, 0x83, 0x2d, 0x2f, 0xf0, 0x64, 0x31, 0x89, 0xc9, 0x1e, 0x50, - 0x84, 0x3f, 0x28, 0xc8, 0x32, 0xc2, 0xe6, 0x65, 0x6d, 0xf6, 0x36, 0xc8, 0x5c, 0x9c, 0xdb, 0x39, - 0xc1, 0x7d, 0x8d, 0x48, 0x65, 0xb3, 0x80, 0x7b, 0x5c, 0x6a, 0x89, 0x49, 0x3d, 0xa4, 0x68, 0x5f, - 0x76, 0x69, 0x10, 0x86, 0x8a, 0x08, 0xf4, 0xaf, 0xf7, 0x61, 0x6e, 0x24, 0x03, 0x5d, 0xd2, 0x56, - 0xad, 0xd2, 0xb6, 0x2a, 0x11, 0xc4, 0x15, 0xbf, 0xad, 0x8a, 0xf3, 0x8e, 0x2a, 0x72, 0xd8, 0xe4, - 0x8c, 0x87, 0x7d, 0x1c, 0x54, 0x59, 0xcc, 0x21, 0xee, 0xf2, 0x98, 0x35, 0x7f, 0x49, 0x81, 0x75, - 0x00, 0xf3, 0x7d, 0x5b, 0x37, 0xba, 0xa4, 0x9e, 0x20, 0xde, 0xe4, 0x19, 0x7d, 0x36, 0x47, 0xce, - 0x6f, 0xfe, 0x20, 0x64, 0x45, 0x43, 0xcf, 0x30, 0xd7, 0x4f, 0xcc, 0xce, 0xfa, 0x91, 0x78, 0x25, - 0xe3, 0x27, 0x92, 0xc3, 0xdc, 0x04, 0x89, 0x9e, 0x42, 0x46, 0x4c, 0x0f, 0xb2, 0xb4, 0x66, 0x9f, - 0x35, 0x8a, 0x89, 0x3a, 0x8b, 0x73, 0xa3, 0x6d, 0x28, 0x59, 0xf8, 0x34, 0x3c, 0xec, 0xca, 0x45, - 0xfc, 0xbc, 0xd0, 0xc2, 0xa7, 0x93, 0x27, 0x5d, 0x05, 0x2b, 0xc0, 0xe8, 0xe8, 0x73, 0x28, 0x46, - 0x52, 0x0c, 0x77, 0x92, 0x19, 0x43, 0xa3, 0x5f, 0x4c, 0x87, 0x32, 0x0f, 0xda, 0x86, 0x8c, 0xc8, - 0x71, 0x79, 0x7a, 0xc6, 0xab, 0x09, 0x13, 0xcc, 0x68, 0x8b, 0xa4, 0xa8, 0x53, 0x2f, 0x48, 0x7d, - 0x85, 0xa0, 0x3c, 0xbe, 0x38, 0x5f, 0xc9, 0x93, 0x13, 0x4e, 0x18, 0x69, 0xe5, 0x2d, 0x1f, 0xae, - 0xa3, 0x5d, 0x00, 0xff, 0x75, 0xd2, 0xa5, 0x43, 0xd7, 0xe9, 0x6d, 0xd2, 0x81, 0x20, 0x0c, 0xb6, - 0xa4, 0x84, 0xb8, 0xd1, 0x33, 0xc8, 0x89, 0x10, 0xc9, 0x1a, 0x98, 0xe9, 0xb1, 0x64, 0x3c, 0x60, - 0x8b, 0x30, 0xed, 0x4b, 0x20, 0xb5, 0x9b, 0x89, 0x35, 0x17, 0xf3, 0x2e, 0xe6, 0xc9, 0x8c, 0xb5, - 0x1b, 0x73, 0xae, 0xda, 0x89, 0x66, 0xf5, 0xf0, 0x3e, 0xe1, 0xdf, 0x8a, 0x97, 0x25, 0x85, 0x89, - 0x42, 0x2d, 0x90, 0xa9, 0xca, 0xc2, 0xf1, 0x5f, 0xa6, 0x5a, 0x7b, 0x57, 0x38, 0x2e, 0xd1, 0xda, - 0xd4, 0x1c, 0x40, 0x6d, 0xea, 0x59, 0x90, 0x07, 0x7e, 0x07, 0x4a, 0x5d, 0xdb, 0xe9, 0x6b, 0x9e, - 0x2a, 0x9c, 0x67, 0x3e, 0x98, 0x77, 0x7c, 0x77, 0xbe, 0x52, 0xdc, 0xa6, 0x58, 0xe1, 0x38, 0xc5, - 0x6e, 0xf8, 0x13, 0xed, 0x88, 0x74, 0x79, 0x8b, 0x66, 0xb7, 0x0f, 0x67, 0x3d, 0xe1, 0x78, 0xae, - 0x6c, 0x41, 0x9a, 0x16, 0xa7, 0x6e, 0x79, 0x81, 0xea, 0xfd, 0x9a, 0x85, 0xae, 0xc2, 0xa5, 0xa0, - 0x7d, 0x28, 0xe9, 0x04, 0x42, 0x3a, 0x27, 0x36, 0x4f, 0xb9, 0x4d, 0xe5, 0xae, 0x4c, 0x91, 0x2b, - 0x02, 0xad, 0x68, 0x9b, 0x05, 0x33, 0x9b, 0xb9, 0xb4, 0x21, 0xdb, 0xd5, 0xfa, 0x86, 0x69, 0x60, - 0xb7, 0xbc, 0x48, 0xe5, 0x7c, 0x74, 0xa9, 0x57, 0x8f, 0xce, 0xd6, 0x45, 0x72, 0x15, 0x42, 0x7c, - 0xe7, 0xa6, 0x80, 0x33, 0x72, 0x89, 0x77, 0xc6, 0x9d, 0x5b, 0xcc, 0xd6, 0x23, 0x73, 0x76, 0xea, - 0xdc, 0xfc, 0x4b, 0x47, 0x0f, 0x00, 0x5e, 0x1b, 0xf8, 0x8d, 0xfa, 0x6a, 0x88, 0x9d, 0xb3, 0x72, - 0x39, 0xdc, 0x26, 0x11, 0xf8, 0xe7, 0x04, 0x8c, 0x3e, 0x86, 0x9c, 0x8e, 0x07, 0xd8, 0xd2, 0xdd, - 0xb6, 0x55, 0xbe, 0x4b, 0x8b, 0xd2, 0x5b, 0x17, 0xe7, 0x2b, 0xb9, 0xba, 0x00, 0xf2, 0x58, 0x1a, - 0x50, 0xa1, 0x2f, 0xa1, 0xc0, 0x3e, 0xb0, 0xde, 0xb6, 0xb6, 0xce, 0xca, 0x4b, 0xf4, 0xd0, 0x8f, - 0x66, 0xbc, 0x94, 0x60, 0x08, 0xe1, 0xcf, 0x6d, 0xeb, 0x21, 0x69, 0x4a, 0x44, 0x36, 0xfa, 0x03, - 0x28, 0x08, 0x6b, 0xde, 0xb5, 0x8f, 0xdd, 0xf2, 0x3b, 0x97, 0x0e, 0x55, 0x47, 0xd7, 0x7a, 0x16, - 0xb0, 0x8a, 0x58, 0x15, 0x96, 0x86, 0x7e, 0x0e, 0x45, 0xff, 0x0d, 0xc7, 0x1e, 0x78, 0x6e, 0xf9, - 0x1e, 0x75, 0xc6, 0xc7, 0xb3, 0x9a, 0x2a, 0xe7, 0x6d, 0x0f, 0xe8, 0xbc, 0x39, 0xf4, 0x85, 0xee, - 0x43, 0x4e, 0x77, 0xec, 0x01, 0xcb, 0x19, 0x3f, 0x58, 0x95, 0xd6, 0x12, 0x7e, 0x27, 0xec, 0xd8, - 0x03, 0x9a, 0x0c, 0x54, 0x28, 0x39, 0x78, 0x60, 0x6a, 0x1d, 0xdc, 0x27, 0xd9, 0xcc, 0xee, 0x96, - 0x97, 0xe9, 0xea, 0x9b, 0x33, 0x2b, 0xd2, 0x67, 0x16, 0x86, 0x19, 0x92, 0xd7, 0xee, 0xa2, 0xe7, - 0x00, 0xda, 0x50, 0x37, 0x3c, 0xb5, 0x6f, 0xeb, 0xb8, 0xbc, 0x72, 0xe9, 0xc3, 0xe2, 0xa8, 0xf0, - 0x2a, 0x61, 0x7c, 0x66, 0xeb, 0xd8, 0x7f, 0xa1, 0x10, 0x00, 0xf4, 0x31, 0xe4, 0xe9, 0xd1, 0xbe, - 0xb4, 0x8f, 0x89, 0x6d, 0xae, 0xd2, 0xc3, 0xcd, 0xf3, 0xbb, 0xcc, 0xd5, 0x1d, 0x7b, 0xb0, 0x6b, - 0x1f, 0x53, 0x8b, 0xe1, 0x7f, 0xea, 0xc8, 0x85, 0x42, 0xaf, 0xa3, 0x06, 0xe1, 0xf3, 0x3e, 0xbd, - 0xc5, 0x4f, 0x67, 0xdc, 0xcb, 0xd3, 0xda, 0x84, 0x80, 0x7a, 0x4b, 0xe4, 0x81, 0xa7, 0x35, 0x01, - 0x73, 0x95, 0x7c, 0xaf, 0xe3, 0x7f, 0x90, 0xbe, 0x83, 0x8d, 0x5b, 0xb8, 0x03, 0x54, 0xc2, 0x7d, - 0x07, 0xc3, 0x30, 0x17, 0x68, 0x01, 0x9f, 0xcb, 0xa8, 0x9a, 0xab, 0xda, 0x5d, 0x76, 0x67, 0x0f, - 0x66, 0xcf, 0xf3, 0x25, 0xc6, 0x5d, 0x75, 0xdb, 0x5d, 0x7a, 0xb1, 0x1d, 0x28, 0xd8, 0x43, 0xef, - 0xd8, 0x1e, 0x5a, 0xba, 0xda, 0x7d, 0xe9, 0x96, 0xdf, 0xa5, 0xa7, 0xbd, 0x52, 0x0f, 0xed, 0x9f, - 0xae, 0xcd, 0x05, 0x6d, 0xef, 0xb9, 0x4a, 0x5e, 0x48, 0xdd, 0x7e, 0xe9, 0xa2, 0x3f, 0x86, 0xbc, - 0x61, 0x05, 0x6b, 0x3c, 0xbc, 0xfa, 0x1a, 0x48, 0xf4, 0x04, 0x4d, 0xcb, 0x5f, 0x02, 0xb8, 0x4c, - 0xb2, 0xc2, 0x07, 0x50, 0xb2, 0xbb, 0x5d, 0xd3, 0xb0, 0xb0, 0xea, 0x60, 0xcd, 0xb5, 0xad, 0xf2, - 0x7b, 0x21, 0x0d, 0x16, 0x39, 0x4e, 0xa1, 0x28, 0x54, 0x81, 0x9c, 0x87, 0xfb, 0x03, 0xdb, 0xd1, - 0x9c, 0xb3, 0xf2, 0xfb, 0xe1, 0x87, 0x1d, 0x1f, 0x8c, 0x8e, 0x61, 0x69, 0x68, 0xe1, 0xd3, 0x81, - 0xed, 0x62, 0x5d, 0x1d, 0xab, 0x30, 0xd7, 0x68, 0x8c, 0x7b, 0xc8, 0x37, 0x75, 0xe7, 0xb9, 0xa0, - 0x9c, 0x58, 0x6a, 0xde, 0x19, 0x4e, 0x44, 0xeb, 0x4b, 0xbf, 0x92, 0x60, 0x7e, 0x2c, 0x47, 0xa2, - 0x3f, 0x82, 0x8c, 0x65, 0xeb, 0xa1, 0x67, 0xb4, 0x06, 0x5f, 0x26, 0xdd, 0xb2, 0x75, 0xf6, 0x8a, - 0xf6, 0xb8, 0x67, 0x78, 0x27, 0xc3, 0xe3, 0xf5, 0x8e, 0xdd, 0xdf, 0xf0, 0x95, 0xa8, 0x1f, 0x07, - 0x7f, 0x6f, 0x0c, 0x5e, 0xf6, 0x36, 0xe8, 0x5f, 0x83, 0xe3, 0x75, 0xc6, 0xa6, 0xa4, 0x89, 0xd4, - 0xa6, 0x8e, 0x3e, 0x82, 0x39, 0x7c, 0x3a, 0x30, 0x9c, 0x50, 0x9d, 0x18, 0x0f, 0xf9, 0x7c, 0x29, - 0x40, 0x12, 0x03, 0xe1, 0x0f, 0x1d, 0xff, 0x10, 0x87, 0xb9, 0x91, 0x0c, 0x45, 0x0a, 0x63, 0xda, - 0x23, 0x47, 0x0a, 0x63, 0x02, 0xb9, 0xa4, 0x5c, 0x0f, 0xff, 0x66, 0x22, 0x71, 0xd3, 0xdf, 0x4c, - 0x44, 0x1f, 0x10, 0x52, 0x57, 0x78, 0x40, 0xf8, 0x29, 0x2c, 0x1a, 0xae, 0x6a, 0xd9, 0x96, 0x98, - 0xf3, 0xf8, 0xed, 0x61, 0xf8, 0x65, 0xfe, 0x96, 0xe1, 0xb6, 0x6c, 0x8b, 0x4d, 0x78, 0xfc, 0x53, - 0x07, 0x8f, 0xf8, 0x99, 0xf1, 0x47, 0x7c, 0x7f, 0x8e, 0x93, 0x94, 0x53, 0x4b, 0x5f, 0x4b, 0x90, - 0x0b, 0xff, 0x12, 0x2c, 0x1e, 0x1d, 0x5d, 0x8c, 0x35, 0x0b, 0xd7, 0x7c, 0x0e, 0x8d, 0x6a, 0x21, - 0x31, 0xbb, 0x16, 0xf8, 0xd5, 0xfe, 0x09, 0xe4, 0x43, 0xa9, 0x67, 0xb4, 0x25, 0x97, 0xae, 0xd1, - 0x92, 0xbf, 0x0b, 0x69, 0x1e, 0x6f, 0x99, 0x61, 0x15, 0x39, 0x77, 0x8a, 0xc5, 0xda, 0xd4, 0x97, - 0x24, 0xce, 0xf2, 0xd5, 0xff, 0x2d, 0x01, 0x85, 0x70, 0x6a, 0x22, 0xce, 0x69, 0x58, 0x1d, 0x87, - 0xe6, 0x05, 0xba, 0x7a, 0xc2, 0x7f, 0x75, 0x15, 0x60, 0x92, 0xb0, 0xfa, 0x86, 0xa5, 0xd2, 0x97, - 0xbe, 0x88, 0xf1, 0x66, 0xfb, 0x86, 0xf5, 0x05, 0x81, 0x52, 0x12, 0xed, 0x94, 0x93, 0x24, 0x22, - 0x24, 0xda, 0x29, 0x23, 0x59, 0xa2, 0x35, 0x9f, 0xe3, 0xd1, 0xc6, 0x2c, 0x11, 0xaa, 0xe2, 0x1c, - 0x0f, 0x2d, 0x43, 0xe6, 0xb5, 0xe1, 0x78, 0x43, 0xcd, 0xa4, 0x3d, 0x98, 0xb8, 0x66, 0x01, 0x44, - 0x16, 0x94, 0x82, 0x64, 0xfc, 0xc6, 0xc2, 0x0e, 0x35, 0x9c, 0xfc, 0x66, 0xf5, 0x1a, 0xd9, 0x38, - 0xf8, 0x20, 0x82, 0x44, 0xc8, 0x72, 0xc3, 0xc0, 0xa5, 0xbf, 0x96, 0xa0, 0x18, 0x21, 0x43, 0x4d, - 0x98, 0xa3, 0x0b, 0x8f, 0xcd, 0xc5, 0xee, 0xfb, 0xbf, 0xe9, 0x22, 0xe8, 0x89, 0x7d, 0x55, 0xd1, - 0x0e, 0xa1, 0x74, 0xf4, 0x19, 0x94, 0x98, 0x28, 0xff, 0xfd, 0x3e, 0x6a, 0x7e, 0x05, 0x2a, 0x29, - 0xfa, 0x88, 0x5f, 0xb0, 0x03, 0x98, 0x1e, 0x7e, 0x9a, 0x5c, 0xb2, 0x20, 0x1f, 0xca, 0xf6, 0x33, - 0xd8, 0xfd, 0x4f, 0x20, 0xe9, 0x47, 0xa1, 0x19, 0xb3, 0x18, 0x65, 0xe0, 0xeb, 0x7d, 0x25, 0xc1, - 0xc2, 0xa4, 0xac, 0x1b, 0xf1, 0x27, 0x66, 0x48, 0x33, 0xf9, 0xd3, 0x83, 0x70, 0x35, 0xc4, 0x8c, - 0x4b, 0x3c, 0x97, 0x05, 0xf5, 0xd0, 0x7b, 0xbe, 0x89, 0x33, 0xdb, 0x9a, 0x8b, 0x98, 0x38, 0xe9, - 0x72, 0x42, 0x46, 0x5e, 0x79, 0x2c, 0xc6, 0x30, 0x00, 0xe9, 0x83, 0xe7, 0x5b, 0xfb, 0xcd, 0xda, - 0xc4, 0x11, 0x0a, 0xca, 0x43, 0xa6, 0xbd, 0xbd, 0xbd, 0xdf, 0x6c, 0x35, 0xe4, 0x44, 0x65, 0x0d, - 0x72, 0x7e, 0x61, 0x83, 0x0a, 0x90, 0xad, 0x37, 0x0f, 0xab, 0x5b, 0xfb, 0x8d, 0xba, 0x1c, 0x43, - 0x45, 0xc8, 0x29, 0x8d, 0x6a, 0x9d, 0x0e, 0x6a, 0x64, 0xe9, 0x93, 0xec, 0x2f, 0xbf, 0x5a, 0x91, - 0x78, 0xe0, 0x49, 0xcb, 0x99, 0xdd, 0x64, 0x16, 0xc9, 0xb7, 0x2a, 0xff, 0x1b, 0x07, 0x54, 0xd7, - 0x3c, 0x8d, 0xd8, 0xdf, 0x15, 0xc6, 0x19, 0xf1, 0x4b, 0x6e, 0x2a, 0xda, 0xa2, 0x26, 0x6e, 0xd4, - 0xa2, 0x4e, 0x1c, 0x58, 0x24, 0x6f, 0x32, 0xb0, 0xb8, 0xd6, 0xdc, 0x64, 0xbc, 0xbd, 0x4a, 0x5f, - 0xbf, 0xbd, 0x0a, 0x6e, 0xa2, 0x72, 0x91, 0x86, 0xd2, 0xd1, 0xd9, 0x20, 0xac, 0xf9, 0x6b, 0x4d, - 0xbe, 0x26, 0xcd, 0xb7, 0xe2, 0x57, 0x9f, 0x6f, 0x5d, 0xf2, 0xfb, 0x61, 0x76, 0xed, 0xc9, 0x4b, - 0xae, 0xbd, 0x0e, 0xc9, 0x97, 0x86, 0xc5, 0xc6, 0xb3, 0xa5, 0xa9, 0x17, 0x1e, 0x3d, 0xed, 0xfa, - 0x9e, 0x61, 0xe9, 0x62, 0x1d, 0xc2, 0x8d, 0x7e, 0x01, 0x05, 0x6c, 0x0d, 0xfb, 0x6a, 0x1f, 0xf7, - 0x8f, 0xb1, 0x23, 0xf4, 0xfc, 0x68, 0x36, 0x69, 0x0d, 0x6b, 0xd8, 0x7f, 0x46, 0x19, 0x45, 0x51, - 0x8c, 0x7d, 0x88, 0x8b, 0x1e, 0x41, 0x4a, 0x33, 0x0d, 0xcd, 0xe5, 0x03, 0xaf, 0xcb, 0x7e, 0xce, - 0xc7, 0x08, 0xd1, 0xef, 0x42, 0x51, 0x73, 0x1c, 0xed, 0x8c, 0xff, 0x2c, 0x4d, 0xa7, 0x63, 0x61, - 0x6e, 0x31, 0xa4, 0x94, 0xad, 0x12, 0x24, 0xfd, 0x25, 0x9a, 0x50, 0x44, 0x5e, 0xf3, 0x41, 0x91, - 0x11, 0x5d, 0xee, 0x66, 0x23, 0x3a, 0xb8, 0x89, 0xc5, 0x8f, 0x1b, 0x6f, 0xfe, 0xfa, 0xc6, 0xbb, - 0xf4, 0x4b, 0x09, 0x20, 0xd0, 0x33, 0xfa, 0x09, 0xdc, 0x19, 0x9c, 0x9c, 0xb9, 0xf4, 0x57, 0x80, - 0x0e, 0x1e, 0x38, 0xd8, 0xc5, 0x16, 0x8b, 0xac, 0xd4, 0x78, 0x0b, 0xca, 0xa2, 0x40, 0x2b, 0x11, - 0x2c, 0xfa, 0x14, 0x16, 0xc5, 0xaf, 0x07, 0x47, 0xf8, 0xc2, 0xf5, 0xe0, 0x6d, 0x4e, 0x13, 0x65, - 0xe6, 0x01, 0xf3, 0x1d, 0x48, 0x12, 0xfb, 0x21, 0xa1, 0xb1, 0xd1, 0x7a, 0xfe, 0x4c, 0x8e, 0xa1, - 0x1c, 0xa4, 0xaa, 0xfb, 0xcd, 0xea, 0x61, 0x38, 0xdc, 0x55, 0xfe, 0x2e, 0x01, 0x32, 0xb3, 0xf0, - 0x9b, 0xba, 0xd9, 0xf4, 0x8a, 0xf5, 0xfb, 0x87, 0xbc, 0xd1, 0xa8, 0x98, 0x7c, 0xfb, 0x51, 0x31, - 0xf5, 0x96, 0xa2, 0x62, 0xfa, 0x06, 0x51, 0x31, 0xf3, 0x56, 0xa2, 0xe2, 0xd7, 0x71, 0x80, 0xd0, - 0x55, 0xfd, 0x2c, 0xfc, 0xbf, 0x23, 0xa6, 0x0f, 0x5f, 0x47, 0x8a, 0xa7, 0x9d, 0x98, 0xf8, 0xbf, - 0x13, 0x4f, 0x21, 0xab, 0xf3, 0x0c, 0xc7, 0xab, 0x87, 0xa9, 0x53, 0xce, 0xb1, 0x44, 0xb8, 0x13, - 0x53, 0x7c, 0x66, 0xf4, 0x69, 0xe4, 0xe7, 0xc0, 0x0f, 0x67, 0x8a, 0x49, 0x3b, 0xe2, 0xa7, 0x14, - 0x55, 0x48, 0xb3, 0xd8, 0xcc, 0xef, 0x7e, 0xda, 0x9c, 0x7c, 0xd4, 0x52, 0x77, 0x62, 0x0a, 0x67, - 0xe4, 0x6f, 0x27, 0x19, 0x48, 0x0d, 0x2d, 0xc3, 0xb6, 0x7e, 0xa4, 0x84, 0x1f, 0xf1, 0x45, 0xfb, - 0x43, 0xb2, 0x3e, 0xfd, 0x5b, 0xf3, 0xb0, 0xce, 0x5e, 0x71, 0x9e, 0x5b, 0xaf, 0x7d, 0x80, 0x84, - 0x4a, 0x00, 0x1c, 0x6f, 0x58, 0x3d, 0x39, 0x4e, 0x6b, 0x06, 0xc7, 0x1e, 0x0c, 0xc8, 0x57, 0x62, - 0xeb, 0x87, 0xdf, 0xfc, 0xe7, 0x72, 0xec, 0x9b, 0x8b, 0x65, 0xe9, 0xd7, 0x17, 0xcb, 0xd2, 0x6f, - 0x2e, 0x96, 0xa5, 0xff, 0xb8, 0x58, 0x96, 0xfe, 0xfc, 0xdb, 0xe5, 0xd8, 0xaf, 0xbf, 0x5d, 0x8e, - 0xfd, 0xe6, 0xdb, 0xe5, 0xd8, 0xef, 0x65, 0xf8, 0x46, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x83, - 0xb6, 0xca, 0xed, 0x81, 0x33, 0x00, 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_994b169dcbedf9d2) +} + +var fileDescriptor_structured_994b169dcbedf9d2 = []byte{ + // 4272 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x7b, 0x49, 0x6c, 0x1b, 0x59, + 0x7a, 0x30, 0xf7, 0xe5, 0xe3, 0x56, 0x7a, 0x96, 0x65, 0x5a, 0xed, 0x96, 0x64, 0xba, 0xdd, 0xad, + 0xe9, 0x45, 0x72, 0xcb, 0xf3, 0xff, 0xe3, 0xe9, 0x4e, 0x06, 0x4d, 0x91, 0x94, 0x45, 0x2d, 0xa4, + 0xba, 0x24, 0xdb, 0x33, 0x99, 0x24, 0x95, 0x12, 0xeb, 0x91, 0xaa, 0x76, 0xb1, 0x8a, 0xae, 0x2a, + 0xda, 0x12, 0x90, 0x53, 0x4e, 0x73, 0x0a, 0x72, 0xc9, 0x31, 0x40, 0x23, 0x69, 0x20, 0x73, 0x4a, + 0x90, 0x4b, 0x72, 0x0b, 0x90, 0x5b, 0xe7, 0x94, 0xc9, 0x6d, 0x4e, 0x42, 0xa2, 0xbe, 0xe4, 0x1e, + 0x20, 0x01, 0x3a, 0x97, 0xe0, 0x6d, 0xb5, 0x70, 0x51, 0x53, 0x92, 0x73, 0x11, 0x58, 0xdf, 0xf6, + 0xb6, 0x6f, 0x7f, 0x4f, 0x70, 0xcf, 0x79, 0x65, 0xac, 0x3b, 0xaf, 0x8c, 0x63, 0xd5, 0xc1, 0xeb, + 0x8e, 0x6b, 0x0f, 0x3b, 0xee, 0xd0, 0xc6, 0xda, 0xda, 0xc0, 0xb6, 0x5c, 0x0b, 0xdd, 0xee, 0x58, + 0x9d, 0x97, 0xb6, 0xa5, 0x76, 0x4e, 0xd6, 0x9c, 0x57, 0xc6, 0x1a, 0xa7, 0x5b, 0x2c, 0x0f, 0x5d, + 0xdd, 0x58, 0x3f, 0x31, 0x3a, 0xeb, 0xae, 0xde, 0xc7, 0x8e, 0xab, 0xf6, 0x07, 0x8c, 0x61, 0xf1, + 0x9d, 0xa0, 0xb8, 0x81, 0xad, 0xbf, 0xd6, 0x0d, 0xdc, 0xc3, 0x1c, 0x79, 0x9b, 0x20, 0xdd, 0xb3, + 0x01, 0x76, 0xd8, 0x5f, 0x0e, 0xbe, 0xdb, 0xc3, 0xd6, 0x7a, 0x0f, 0x5b, 0xba, 0xa9, 0xe1, 0xd3, + 0xf5, 0x8e, 0x65, 0x76, 0xf5, 0x1e, 0x47, 0xcd, 0xf7, 0xac, 0x9e, 0x45, 0x7f, 0xae, 0x93, 0x5f, + 0x0c, 0x5a, 0xf9, 0x93, 0x24, 0xdc, 0xda, 0xb2, 0x6c, 0xac, 0xf7, 0xcc, 0x5d, 0x7c, 0x26, 0xe3, + 0x2e, 0xb6, 0xb1, 0xd9, 0xc1, 0x68, 0x05, 0x92, 0xae, 0x7a, 0x6c, 0xe0, 0x72, 0x74, 0x25, 0xba, + 0x5a, 0xd8, 0x84, 0x6f, 0xcf, 0x97, 0x23, 0xdf, 0x9f, 0x2f, 0xc7, 0x9a, 0x75, 0x99, 0x21, 0xd0, + 0x43, 0x48, 0xd2, 0x51, 0xca, 0x31, 0x4a, 0x51, 0xe2, 0x14, 0xe9, 0x26, 0x01, 0x12, 0x32, 0x8a, + 0x45, 0x65, 0x48, 0x98, 0x6a, 0x1f, 0x97, 0xe3, 0x2b, 0xd1, 0xd5, 0xec, 0x66, 0x82, 0x50, 0xc9, + 0x14, 0x82, 0x76, 0x21, 0xf3, 0x5a, 0x35, 0x74, 0x4d, 0x77, 0xcf, 0xca, 0x89, 0x95, 0xe8, 0x6a, + 0x71, 0xe3, 0x47, 0x6b, 0x13, 0xf7, 0x68, 0xad, 0x66, 0x99, 0x8e, 0x6b, 0xab, 0xba, 0xe9, 0x3e, + 0xe7, 0x0c, 0x5c, 0x90, 0x27, 0x00, 0x3d, 0x82, 0x39, 0xe7, 0x44, 0xb5, 0xb1, 0xa6, 0x0c, 0x6c, + 0xdc, 0xd5, 0x4f, 0x15, 0x03, 0x9b, 0xe5, 0xe4, 0x4a, 0x74, 0x35, 0xc9, 0x49, 0x4b, 0x0c, 0x7d, + 0x40, 0xb1, 0x7b, 0xd8, 0x44, 0x47, 0x90, 0xb5, 0x4c, 0x45, 0xc3, 0x06, 0x76, 0x71, 0x39, 0x45, + 0xc7, 0xff, 0x74, 0xca, 0xf8, 0x13, 0x36, 0x68, 0xad, 0xda, 0x71, 0x75, 0xcb, 0x14, 0xf3, 0xb0, + 0xcc, 0x3a, 0x15, 0xc4, 0xa5, 0x0e, 0x07, 0x9a, 0xea, 0xe2, 0x72, 0xfa, 0xc6, 0x52, 0x9f, 0x51, + 0x41, 0x68, 0x0f, 0x92, 0x7d, 0xd5, 0xed, 0x9c, 0x94, 0x33, 0x54, 0xe2, 0xa3, 0x2b, 0x48, 0xdc, + 0x27, 0x7c, 0x5c, 0x20, 0x13, 0x52, 0x79, 0x01, 0x29, 0x36, 0x0e, 0x2a, 0x40, 0xb6, 0xd5, 0x56, + 0xaa, 0xb5, 0xa3, 0x66, 0xbb, 0x25, 0x45, 0x50, 0x1e, 0x32, 0x72, 0xe3, 0xf0, 0x48, 0x6e, 0xd6, + 0x8e, 0xa4, 0x28, 0xf9, 0x3a, 0x6c, 0x1c, 0x29, 0xad, 0x67, 0x7b, 0x7b, 0x52, 0x0c, 0x95, 0x20, + 0x47, 0xbe, 0xea, 0x8d, 0xad, 0xea, 0xb3, 0xbd, 0x23, 0x29, 0x8e, 0x72, 0x90, 0xae, 0x55, 0x0f, + 0x6b, 0xd5, 0x7a, 0x43, 0x4a, 0x2c, 0x26, 0x7e, 0xfd, 0xcd, 0x52, 0xa4, 0xf2, 0x08, 0x92, 0x74, + 0x38, 0x04, 0x90, 0x3a, 0x6c, 0xee, 0x1f, 0xec, 0x35, 0xa4, 0x08, 0xca, 0x40, 0x62, 0x8b, 0x88, + 0x88, 0x12, 0x8e, 0x83, 0xaa, 0x7c, 0xd4, 0xac, 0xee, 0x49, 0x31, 0xc6, 0xf1, 0x59, 0xe2, 0x3f, + 0xbe, 0x5e, 0x8e, 0x56, 0xfe, 0x35, 0x09, 0xf3, 0xfe, 0xdc, 0xfd, 0xd3, 0x46, 0x35, 0x28, 0x59, + 0xb6, 0xde, 0xd3, 0x4d, 0x85, 0xea, 0x9c, 0xa2, 0x6b, 0x5c, 0x1f, 0xdf, 0x21, 0xeb, 0xb9, 0x38, + 0x5f, 0x2e, 0xb4, 0x29, 0xfa, 0x88, 0x60, 0x9b, 0x75, 0xae, 0xa0, 0x05, 0x2b, 0x00, 0xd4, 0xd0, + 0x2e, 0xcc, 0x71, 0x21, 0x1d, 0xcb, 0x18, 0xf6, 0x4d, 0x45, 0xd7, 0x9c, 0x72, 0x6c, 0x25, 0xbe, + 0x5a, 0xd8, 0x5c, 0xbe, 0x38, 0x5f, 0x2e, 0x31, 0x11, 0x35, 0x8a, 0x6b, 0xd6, 0x9d, 0xef, 0xcf, + 0x97, 0x33, 0xe2, 0x43, 0xe6, 0xc3, 0xf3, 0x6f, 0xcd, 0x41, 0x2f, 0xe0, 0xb6, 0x2d, 0xf6, 0x56, + 0x0b, 0x0a, 0x8c, 0x53, 0x81, 0x0f, 0x2e, 0xce, 0x97, 0x6f, 0x79, 0x9b, 0xaf, 0x4d, 0x16, 0x7a, + 0xcb, 0x1e, 0x25, 0xd0, 0x1c, 0xd4, 0x86, 0x00, 0xd8, 0x5f, 0x6e, 0x82, 0x2e, 0x77, 0x99, 0x2f, + 0x77, 0xce, 0x17, 0x1d, 0x5e, 0xf2, 0x9c, 0x3d, 0x82, 0xd0, 0x3c, 0xc3, 0x4b, 0x5e, 0x6a, 0x78, + 0xa9, 0x9b, 0x1a, 0x5e, 0xc8, 0x8c, 0xd2, 0xff, 0x27, 0x66, 0x94, 0x79, 0xeb, 0x66, 0x94, 0x7d, + 0x0b, 0x66, 0xc4, 0x74, 0x77, 0x27, 0x91, 0x01, 0x29, 0xb7, 0x93, 0xc8, 0xe4, 0xa4, 0xfc, 0x4e, + 0x22, 0x93, 0x97, 0x0a, 0x3b, 0x89, 0x4c, 0x41, 0x2a, 0x56, 0xfe, 0x33, 0x01, 0x12, 0x3b, 0xdd, + 0x3a, 0x76, 0x3a, 0xb6, 0x3e, 0x70, 0x2d, 0xdb, 0x3b, 0x93, 0xe8, 0xd8, 0x99, 0xbc, 0x0f, 0x31, + 0x5d, 0xe3, 0xae, 0x74, 0x81, 0x9f, 0x76, 0x8c, 0x1e, 0xaf, 0xaf, 0x37, 0x31, 0x5d, 0x43, 0x6b, + 0x90, 0x20, 0xfe, 0x9e, 0xba, 0xd3, 0xdc, 0xc6, 0xe2, 0xe8, 0x0a, 0x70, 0x7f, 0x8d, 0x85, 0x83, + 0x23, 0x99, 0xd2, 0xa1, 0x15, 0xc8, 0x98, 0x43, 0xc3, 0xa0, 0xae, 0x9c, 0xe8, 0x52, 0x46, 0x6c, + 0x8a, 0x80, 0xa2, 0xfb, 0x90, 0xd7, 0x70, 0x57, 0x1d, 0x1a, 0xae, 0x82, 0x4f, 0x07, 0x36, 0xd3, + 0x17, 0x39, 0xc7, 0x61, 0x8d, 0xd3, 0x81, 0x8d, 0xee, 0x41, 0xea, 0x44, 0xd7, 0x34, 0x6c, 0x52, + 0x75, 0x11, 0x22, 0x38, 0x0c, 0x6d, 0xc0, 0xdc, 0xd0, 0xc1, 0x8e, 0xe2, 0xe0, 0x57, 0x43, 0xb2, + 0x57, 0xd4, 0x1c, 0x80, 0x9a, 0x43, 0x8a, 0xab, 0x67, 0x89, 0x10, 0x1c, 0x72, 0x3c, 0xd1, 0xf6, + 0xfb, 0x90, 0xef, 0x58, 0xfd, 0xc1, 0xd0, 0xc5, 0x6c, 0xd0, 0x1c, 0x1b, 0x94, 0xc3, 0xe8, 0xa0, + 0x1b, 0x30, 0x67, 0xbd, 0x31, 0x47, 0xc4, 0xe6, 0xc3, 0x62, 0x09, 0x41, 0x50, 0xec, 0x26, 0x80, + 0x61, 0xf5, 0xf4, 0x8e, 0x6a, 0x10, 0xdb, 0x29, 0xd0, 0xdd, 0x7c, 0xc0, 0x77, 0xb3, 0xb4, 0xc7, + 0x30, 0x62, 0x3b, 0x43, 0x5b, 0x9b, 0xe5, 0x6c, 0x4d, 0x0d, 0x6d, 0xc1, 0xbb, 0xaa, 0xe1, 0x62, + 0x5b, 0x18, 0x37, 0xd9, 0x46, 0x45, 0x37, 0x95, 0x81, 0x6d, 0xf5, 0x6c, 0xec, 0x38, 0xe5, 0x62, + 0x60, 0x0f, 0xee, 0x52, 0x52, 0x26, 0xe6, 0xe8, 0x6c, 0x80, 0x9b, 0xe6, 0x01, 0x27, 0x43, 0xbf, + 0x04, 0xe4, 0x9c, 0x39, 0x2e, 0xee, 0x0b, 0x41, 0x2f, 0x75, 0x53, 0x2b, 0x97, 0xa8, 0xe6, 0x7d, + 0x30, 0x45, 0xf3, 0x0e, 0x29, 0x03, 0x13, 0xb7, 0xab, 0x9b, 0x1a, 0x1f, 0x45, 0x72, 0x46, 0xe0, + 0x9e, 0xee, 0x65, 0xa4, 0xec, 0x4e, 0x22, 0x93, 0x95, 0x60, 0x27, 0x91, 0x49, 0x4b, 0x99, 0xca, + 0x9f, 0xc6, 0x60, 0x81, 0x91, 0x6d, 0xa9, 0x7d, 0xdd, 0x38, 0xbb, 0xa9, 0xee, 0x31, 0x29, 0x5c, + 0xf7, 0xe8, 0xa1, 0xd1, 0xa5, 0x10, 0x36, 0xe6, 0xf2, 0xe8, 0xa1, 0x11, 0x58, 0x8b, 0x80, 0xd0, + 0x13, 0x80, 0x80, 0x4f, 0x4c, 0xd0, 0xd3, 0xba, 0x7b, 0x71, 0xbe, 0x9c, 0x9d, 0xec, 0x09, 0xb3, + 0x9d, 0x80, 0xff, 0x9b, 0x13, 0x6a, 0xe8, 0x49, 0xa0, 0xba, 0x18, 0x38, 0xc1, 0x3a, 0x23, 0x98, + 0x78, 0x82, 0x25, 0x2d, 0x84, 0xe4, 0x5b, 0x54, 0xf9, 0x87, 0x18, 0xcc, 0x37, 0x4d, 0x17, 0xdb, + 0x06, 0x56, 0x5f, 0xe3, 0xc0, 0x76, 0xfc, 0x1c, 0xb2, 0xaa, 0xd9, 0xc1, 0x8e, 0x6b, 0xd9, 0x4e, + 0x39, 0xba, 0x12, 0x5f, 0xcd, 0x6d, 0xfc, 0x78, 0xca, 0xa9, 0x4c, 0xe2, 0x5f, 0xab, 0x72, 0x66, + 0xbe, 0x93, 0xbe, 0xb0, 0xc5, 0x7f, 0x8c, 0x42, 0x46, 0x60, 0xd1, 0x23, 0xc8, 0x8c, 0x84, 0xae, + 0xdb, 0x7c, 0x35, 0xe9, 0xb0, 0x07, 0x4f, 0xbb, 0xdc, 0x6f, 0xff, 0x3f, 0xc8, 0xd0, 0xcc, 0x49, + 0xf1, 0xce, 0x64, 0x51, 0x70, 0xf0, 0xd4, 0x2a, 0x98, 0x65, 0xa5, 0x29, 0x6d, 0x53, 0x43, 0xb5, + 0x49, 0x09, 0x50, 0x9c, 0xf2, 0xdf, 0x11, 0xfb, 0x77, 0x18, 0x4e, 0x81, 0xc6, 0x72, 0x22, 0xb6, + 0x67, 0x7c, 0xe7, 0xfe, 0x3e, 0x0a, 0x73, 0x84, 0x41, 0xc3, 0x5a, 0x60, 0xdb, 0x1e, 0x00, 0xe8, + 0x8e, 0xe2, 0x30, 0x38, 0x5d, 0x91, 0x30, 0x85, 0xac, 0xee, 0x70, 0x72, 0x4f, 0xd5, 0x62, 0x63, + 0xaa, 0xf6, 0x53, 0x28, 0x50, 0x5e, 0xe5, 0x78, 0xd8, 0x79, 0x89, 0x5d, 0x87, 0xce, 0x30, 0xb9, + 0x39, 0xcf, 0x67, 0x98, 0xa7, 0x12, 0x36, 0x19, 0x4e, 0xce, 0x3b, 0x81, 0xaf, 0x31, 0xed, 0x4b, + 0x8c, 0x69, 0x1f, 0x9f, 0xf8, 0x7f, 0xc7, 0x61, 0xe1, 0x40, 0xb5, 0x5d, 0x9d, 0xc4, 0x00, 0xdd, + 0xec, 0x05, 0x66, 0xff, 0x10, 0x72, 0xe6, 0x50, 0x18, 0xa4, 0xc3, 0x0f, 0x84, 0xcd, 0x0f, 0xcc, + 0x21, 0x37, 0x30, 0x07, 0xed, 0x41, 0xc2, 0xd0, 0x1d, 0x97, 0x26, 0x09, 0xb9, 0x8d, 0x8d, 0x29, + 0x6a, 0x31, 0x79, 0x8c, 0xb5, 0x3d, 0xdd, 0x71, 0xc5, 0x9a, 0x89, 0x14, 0xd4, 0x86, 0xa4, 0xad, + 0x9a, 0x3d, 0x4c, 0xed, 0x25, 0xb7, 0xf1, 0xf8, 0x6a, 0xe2, 0x64, 0xc2, 0x2a, 0x02, 0x0f, 0x95, + 0xb3, 0xf8, 0x17, 0x51, 0x48, 0x90, 0x51, 0x2e, 0x31, 0xe9, 0x05, 0x48, 0xbd, 0x56, 0x8d, 0x21, + 0x66, 0x89, 0x4e, 0x5e, 0xe6, 0x5f, 0xe8, 0x0f, 0xa0, 0xe4, 0x0c, 0x8f, 0x07, 0x81, 0xa1, 0x78, + 0x24, 0xf9, 0xe4, 0x4a, 0xb3, 0xf2, 0x72, 0xea, 0xb0, 0x2c, 0x76, 0x00, 0x8b, 0xaf, 0x20, 0x49, + 0x67, 0x7d, 0xc9, 0xfc, 0xee, 0x43, 0xde, 0xb5, 0x14, 0x7c, 0xda, 0x31, 0x86, 0x8e, 0xfe, 0x9a, + 0x69, 0x4a, 0x5e, 0xce, 0xb9, 0x56, 0x43, 0x80, 0xd0, 0x43, 0x28, 0x76, 0x6d, 0xab, 0xaf, 0xe8, + 0xa6, 0x20, 0x8a, 0x53, 0xa2, 0x02, 0x81, 0x36, 0x05, 0x30, 0xa4, 0xb2, 0x7f, 0x9e, 0x87, 0x12, + 0x35, 0x8c, 0x99, 0xdc, 0xde, 0xc3, 0x80, 0xdb, 0xbb, 0x1d, 0x72, 0x7b, 0x9e, 0x75, 0x11, 0xaf, + 0x77, 0x0f, 0x52, 0x43, 0x53, 0x7f, 0x35, 0x64, 0xe3, 0x7b, 0xc1, 0x8f, 0xc1, 0x66, 0xd0, 0x4a, + 0xf4, 0x31, 0x20, 0xe2, 0x0a, 0xb0, 0x12, 0x22, 0x4c, 0x52, 0x42, 0x89, 0x62, 0x6a, 0x53, 0x3d, + 0x68, 0xea, 0x0a, 0x1e, 0x74, 0x1b, 0x24, 0x7c, 0xea, 0xda, 0x6a, 0x30, 0x2b, 0x4d, 0x53, 0xfe, + 0xa5, 0x8b, 0xf3, 0xe5, 0x62, 0x83, 0xe0, 0x26, 0x0b, 0x29, 0xe2, 0x00, 0x4e, 0x23, 0x5a, 0x32, + 0xc7, 0x65, 0x68, 0xba, 0x8d, 0x69, 0x2e, 0xe5, 0x94, 0x33, 0x2b, 0xf1, 0x4b, 0x72, 0xa6, 0x91, + 0x6d, 0x5f, 0xab, 0x0b, 0x46, 0x59, 0x62, 0xa2, 0x3c, 0x80, 0x83, 0x0e, 0x21, 0xd7, 0x65, 0x29, + 0x96, 0xf2, 0x12, 0x9f, 0xd1, 0x64, 0x2c, 0xb7, 0xf1, 0xe1, 0xec, 0xc9, 0xd8, 0x66, 0x8a, 0x1c, + 0x41, 0x39, 0x2a, 0x43, 0xd7, 0x43, 0xa2, 0x17, 0x50, 0x08, 0xe4, 0xcf, 0xc7, 0x67, 0x34, 0x03, + 0xb9, 0x9e, 0xd8, 0xbc, 0x2f, 0x68, 0xf3, 0x0c, 0x7d, 0x09, 0xa0, 0x7b, 0x01, 0x80, 0x26, 0x2a, + 0xb9, 0x8d, 0x8f, 0xae, 0x10, 0x29, 0x84, 0x7f, 0xf1, 0x85, 0xa0, 0x17, 0x50, 0xf4, 0xbf, 0xe8, + 0x64, 0xf3, 0x57, 0x9e, 0x2c, 0x93, 0x5a, 0x08, 0xc8, 0xd9, 0x24, 0xc9, 0xf8, 0x3c, 0x49, 0xa1, + 0x2c, 0x47, 0x77, 0x71, 0x50, 0x0d, 0x0a, 0x54, 0x0d, 0x2a, 0x17, 0xe7, 0xcb, 0xa8, 0x26, 0xf0, + 0x93, 0x55, 0x01, 0x75, 0x46, 0xf0, 0x4c, 0xb1, 0x42, 0x0a, 0x4c, 0x24, 0x16, 0x7d, 0xc5, 0x3a, + 0xf4, 0x55, 0x78, 0x4c, 0xb1, 0x02, 0xea, 0xcd, 0xaa, 0xa7, 0x7c, 0xc8, 0xf7, 0x94, 0xae, 0xef, + 0x7b, 0x42, 0x82, 0x50, 0x83, 0xa7, 0xc5, 0x12, 0x4d, 0xaf, 0x3e, 0x9a, 0x51, 0x49, 0x49, 0xc6, + 0x26, 0x5c, 0x02, 0xcd, 0x96, 0x1f, 0x03, 0xea, 0xd8, 0x58, 0x75, 0xb1, 0x46, 0xd2, 0x52, 0x43, + 0xef, 0xe8, 0xae, 0x71, 0x56, 0x9e, 0x0b, 0xd8, 0xfd, 0x1c, 0xc7, 0x37, 0x3c, 0x34, 0x7a, 0x02, + 0xe9, 0xd7, 0xd8, 0x76, 0x74, 0xcb, 0x2c, 0x23, 0xea, 0x4c, 0x96, 0x78, 0x2b, 0x64, 0x61, 0x64, + 0xbc, 0xe7, 0x8c, 0x4a, 0x16, 0xe4, 0x68, 0x1b, 0x0a, 0xd8, 0xec, 0x58, 0x9a, 0x6e, 0xf6, 0x68, + 0x9a, 0x59, 0xbe, 0xe5, 0xe7, 0x3b, 0xdf, 0x9f, 0x2f, 0xbf, 0x33, 0xc2, 0xdf, 0xe0, 0xb4, 0x64, + 0xda, 0x72, 0x1e, 0x07, 0xbe, 0xd0, 0x36, 0xa4, 0x45, 0x4c, 0x9e, 0xa7, 0x7b, 0xba, 0x3a, 0x2d, + 0xc3, 0x1c, 0x8d, 0xe8, 0x7c, 0x5d, 0x82, 0x9d, 0x14, 0x0c, 0x9a, 0xee, 0x90, 0x5c, 0x44, 0x2b, + 0xdf, 0x0e, 0x16, 0x0c, 0x02, 0x8a, 0x6a, 0x00, 0x3d, 0x6c, 0x29, 0xac, 0xb9, 0x54, 0x5e, 0xa0, + 0xc3, 0x2d, 0x05, 0x86, 0xeb, 0x61, 0x6b, 0x4d, 0xb4, 0xa0, 0x48, 0x05, 0xd9, 0xd5, 0x7b, 0x22, + 0x45, 0xe8, 0x61, 0x8b, 0x01, 0x50, 0x05, 0xb2, 0x03, 0x1b, 0x6b, 0x7a, 0x87, 0x14, 0x78, 0x77, + 0x02, 0xbe, 0xd9, 0x07, 0x57, 0x96, 0x20, 0xeb, 0x79, 0x0d, 0x94, 0x86, 0x78, 0xf5, 0xb0, 0xc6, + 0xfa, 0x09, 0xf5, 0xc6, 0x61, 0x4d, 0x8a, 0x56, 0xee, 0x43, 0x82, 0x2e, 0x3e, 0x07, 0xe9, 0xad, + 0xb6, 0xfc, 0xa2, 0x2a, 0xd7, 0x59, 0x0f, 0xa3, 0xd9, 0x7a, 0xde, 0x90, 0x8f, 0x1a, 0x75, 0x49, + 0xc4, 0x85, 0x7f, 0x8a, 0x03, 0xf2, 0x4b, 0xd9, 0x23, 0x8b, 0x97, 0x83, 0x3d, 0x28, 0x75, 0x3c, + 0x28, 0x3b, 0x80, 0xe8, 0x4a, 0x6c, 0xb5, 0xb8, 0xf1, 0xe4, 0x07, 0xcb, 0x61, 0x21, 0x23, 0x08, + 0xf2, 0x95, 0xa9, 0xd8, 0x09, 0x41, 0x03, 0xf9, 0x50, 0x6c, 0x24, 0x06, 0xc9, 0x90, 0xec, 0x9c, + 0xe0, 0xce, 0x4b, 0x1e, 0x85, 0xff, 0xff, 0x94, 0x81, 0x69, 0xaa, 0x18, 0x50, 0xdc, 0x1a, 0xe1, + 0xf1, 0x87, 0x16, 0xe9, 0x01, 0x15, 0x85, 0xe4, 0xb0, 0x7b, 0x4d, 0x5c, 0xea, 0xb1, 0x26, 0xb5, + 0x5d, 0x84, 0xc7, 0x0a, 0x78, 0xd7, 0x27, 0x50, 0x32, 0x2d, 0x57, 0x21, 0x45, 0x23, 0xf7, 0x02, + 0xb4, 0x14, 0x2c, 0x6c, 0x4a, 0x5c, 0x57, 0x7d, 0x9b, 0x2f, 0x98, 0x96, 0xdb, 0x1a, 0x1a, 0xbc, + 0xce, 0xaa, 0x7c, 0x06, 0xc5, 0xf0, 0x1e, 0xa1, 0x2c, 0x24, 0x6b, 0xdb, 0x8d, 0xda, 0xae, 0x14, + 0x41, 0x25, 0xc8, 0x6d, 0xb5, 0xe5, 0x46, 0xf3, 0x69, 0x4b, 0xd9, 0x6d, 0xfc, 0x82, 0xf5, 0x9c, + 0x5a, 0x6d, 0xd1, 0x73, 0xf2, 0xaa, 0x9c, 0xa4, 0x94, 0xaa, 0xfc, 0x57, 0x14, 0x8a, 0x07, 0xb6, + 0xde, 0x57, 0xed, 0xb3, 0x5d, 0x7c, 0x76, 0xf8, 0x46, 0x1d, 0xa0, 0x2f, 0x60, 0xde, 0xc4, 0x6f, + 0x94, 0x01, 0x83, 0x2a, 0x5e, 0xd6, 0x1c, 0x9d, 0xdc, 0x90, 0x9c, 0x33, 0xf1, 0x1b, 0x2e, 0xa1, + 0xc9, 0x93, 0xe6, 0x8f, 0x21, 0x67, 0x19, 0x1a, 0xe3, 0xc4, 0xa2, 0x29, 0x94, 0x0b, 0x32, 0x81, + 0x65, 0x68, 0x4d, 0x86, 0x26, 0xd4, 0x64, 0x3c, 0x41, 0x1d, 0x9f, 0x40, 0x6d, 0xe2, 0x37, 0x82, + 0xfa, 0x0b, 0x98, 0x27, 0xb2, 0xc7, 0x66, 0x97, 0x98, 0x32, 0x3b, 0xcb, 0xd0, 0xc2, 0xb3, 0xe3, + 0xca, 0xfb, 0xb7, 0x51, 0xa0, 0x4e, 0x7b, 0xe8, 0x8a, 0x76, 0x11, 0x5d, 0xfc, 0x8f, 0xa1, 0x40, + 0x26, 0xe3, 0xd7, 0x4a, 0xd1, 0x29, 0xe7, 0x41, 0xe6, 0x2c, 0x3c, 0x30, 0xe1, 0x22, 0x93, 0xf2, + 0xb9, 0x62, 0xd3, 0xb8, 0x2c, 0xc3, 0x6b, 0x4e, 0xa1, 0x0f, 0x20, 0xaf, 0x9b, 0xc4, 0x69, 0xf1, + 0x6a, 0x3d, 0xd8, 0xcb, 0xcd, 0x71, 0x0c, 0xa9, 0xd9, 0xf9, 0x8c, 0xff, 0x2a, 0x05, 0xc8, 0x57, + 0xd6, 0xfd, 0xa1, 0xab, 0x52, 0x0b, 0xae, 0x42, 0x8a, 0xab, 0x4e, 0x94, 0xaa, 0xe4, 0x07, 0x53, + 0xad, 0x2c, 0xdc, 0x35, 0xd9, 0x8e, 0xc8, 0x9c, 0x11, 0xfd, 0x2c, 0xd8, 0x73, 0xce, 0x6d, 0xbc, + 0x3f, 0x9b, 0x9f, 0xdf, 0x8e, 0x88, 0x66, 0xf4, 0x2e, 0x24, 0x1d, 0x97, 0x78, 0x9c, 0x38, 0x8d, + 0x13, 0xeb, 0x53, 0xf8, 0xc7, 0x27, 0xbf, 0x76, 0x48, 0xd8, 0x84, 0x9d, 0x51, 0x19, 0xe8, 0x05, + 0x64, 0xbd, 0xf4, 0x88, 0x37, 0xb0, 0x1f, 0xcf, 0x2e, 0xd0, 0xf3, 0x6c, 0xc2, 0xef, 0x79, 0xb2, + 0x50, 0x15, 0x72, 0x7d, 0x4e, 0xe6, 0x17, 0xc1, 0x2b, 0x3c, 0x43, 0x05, 0x21, 0x81, 0x66, 0xaa, + 0x81, 0x2f, 0x19, 0x04, 0x53, 0x93, 0x7a, 0x71, 0xdb, 0x32, 0x8c, 0x63, 0xb5, 0xf3, 0x92, 0x36, + 0xe5, 0x3c, 0x2f, 0x2e, 0xa0, 0x68, 0x97, 0xe4, 0x99, 0xc2, 0x2e, 0x69, 0x8b, 0x2d, 0x37, 0x43, + 0x1b, 0x50, 0xf8, 0xbd, 0xed, 0x88, 0x1c, 0x60, 0x47, 0x6d, 0x28, 0x0e, 0x42, 0xb6, 0xc9, 0x93, + 0xba, 0x87, 0xd3, 0x22, 0x7b, 0x88, 0x78, 0x3b, 0x22, 0x8f, 0xb0, 0xa3, 0x5f, 0x02, 0xea, 0x8c, + 0xe9, 0x7c, 0x19, 0x7e, 0x60, 0x96, 0xa3, 0x0c, 0xdb, 0x11, 0x79, 0x82, 0x98, 0xca, 0x17, 0x90, + 0xa4, 0xc7, 0x49, 0x02, 0xc7, 0xb3, 0xd6, 0x6e, 0xab, 0xfd, 0xa2, 0xc5, 0x7c, 0x51, 0xbd, 0xb1, + 0xd7, 0x38, 0x6a, 0x28, 0xed, 0xd6, 0x1e, 0xf1, 0x45, 0x77, 0xe1, 0x36, 0x07, 0x54, 0x5b, 0x75, + 0xe5, 0x85, 0xdc, 0x14, 0xa8, 0x58, 0x65, 0x35, 0x18, 0x99, 0x32, 0x90, 0x68, 0xb5, 0x5b, 0x0d, + 0x29, 0x42, 0x63, 0x54, 0xbd, 0x2e, 0x45, 0x69, 0x8c, 0x92, 0xdb, 0x07, 0xc2, 0x85, 0x6d, 0xe6, + 0x01, 0x34, 0x4f, 0x05, 0x76, 0x12, 0x99, 0x94, 0x94, 0xae, 0xfc, 0x75, 0x14, 0x32, 0x24, 0xe5, + 0x6f, 0x9a, 0x5d, 0x0b, 0x3d, 0x86, 0xec, 0x40, 0xb5, 0xb1, 0xe9, 0xfa, 0x96, 0x2c, 0x3a, 0x31, + 0x99, 0x03, 0x8a, 0xf0, 0x1a, 0x05, 0x19, 0x46, 0xd8, 0xbc, 0xac, 0xcc, 0xde, 0x02, 0x89, 0x8b, + 0x73, 0x3a, 0x27, 0xb8, 0xaf, 0x12, 0xa9, 0xac, 0x17, 0x70, 0x8f, 0x4b, 0x2d, 0x32, 0xa9, 0x87, + 0x14, 0xed, 0xc9, 0x2e, 0x0e, 0x82, 0x50, 0xe1, 0x81, 0xfe, 0xf9, 0x3e, 0x94, 0x46, 0x22, 0xd0, + 0x25, 0x65, 0xd5, 0x0a, 0x2d, 0xab, 0xe2, 0xbe, 0x5f, 0xf1, 0xca, 0xaa, 0x18, 0xaf, 0xa8, 0x42, + 0x8b, 0x4d, 0xcc, 0xb8, 0xd8, 0xc7, 0x7e, 0x96, 0xc5, 0x0c, 0xe2, 0x2e, 0xf7, 0x59, 0x73, 0x97, + 0x24, 0x58, 0x07, 0x30, 0xd7, 0xb7, 0x34, 0xbd, 0x4b, 0xf2, 0x09, 0x62, 0x4d, 0xae, 0xde, 0x67, + 0x4d, 0xea, 0xdc, 0xc6, 0xbb, 0x01, 0x2d, 0x1a, 0xba, 0xba, 0xb1, 0x76, 0x62, 0x74, 0xd6, 0x8e, + 0xc4, 0x15, 0x9c, 0x68, 0xbc, 0x05, 0xb9, 0x09, 0x12, 0x3d, 0x85, 0xb4, 0xe8, 0x1e, 0x64, 0x68, + 0xce, 0x3e, 0xab, 0x17, 0x13, 0x79, 0x16, 0xe7, 0x46, 0x5b, 0x50, 0x34, 0xf1, 0x69, 0xb0, 0xd9, + 0x95, 0x0d, 0xd9, 0x79, 0xbe, 0x85, 0x4f, 0x27, 0x77, 0xba, 0xf2, 0xa6, 0x8f, 0xd1, 0xd0, 0x97, + 0x50, 0x08, 0x85, 0x18, 0x6e, 0x24, 0x33, 0xba, 0x46, 0x2f, 0x99, 0x0e, 0x44, 0x1e, 0xb4, 0x05, + 0x69, 0x11, 0xe3, 0x72, 0x74, 0x8d, 0x57, 0x13, 0x26, 0x98, 0xd1, 0x26, 0x09, 0x51, 0xa7, 0xae, + 0x1f, 0xfa, 0xf2, 0x7e, 0x7a, 0x7c, 0x71, 0xbe, 0x9c, 0x23, 0x2b, 0x9c, 0xd0, 0xd2, 0xca, 0x99, + 0x1e, 0x5c, 0x43, 0x3b, 0x00, 0xde, 0xd5, 0xa7, 0x43, 0x3b, 0xba, 0xd3, 0xcb, 0xa4, 0x03, 0x41, + 0xe8, 0x4f, 0x49, 0x0e, 0x70, 0xa3, 0x7d, 0xc8, 0x0a, 0x17, 0xc9, 0x0a, 0x98, 0xe9, 0xbe, 0x64, + 0xdc, 0x61, 0x0b, 0x37, 0xed, 0x49, 0x20, 0xb9, 0x9b, 0x81, 0x55, 0x07, 0xf3, 0x2a, 0xe6, 0xc9, + 0x8c, 0xb9, 0x1b, 0x33, 0xae, 0xda, 0x89, 0x6a, 0xf6, 0xf0, 0x1e, 0xe1, 0xdf, 0x8c, 0x95, 0xa3, + 0x32, 0x13, 0x85, 0x5a, 0x20, 0xd1, 0x2d, 0x0b, 0xfa, 0x7f, 0x89, 0xee, 0xda, 0x7b, 0xc2, 0x70, + 0xc9, 0xae, 0x4d, 0x8d, 0x01, 0x54, 0xa7, 0xf6, 0xfd, 0x38, 0xf0, 0x3b, 0x50, 0xec, 0x5a, 0x76, + 0x5f, 0x75, 0x15, 0x61, 0x3c, 0x73, 0x7e, 0xbf, 0xe3, 0xfb, 0xf3, 0xe5, 0xc2, 0x16, 0xc5, 0x0a, + 0xc3, 0x29, 0x74, 0x83, 0x9f, 0x68, 0x5b, 0x84, 0xcb, 0x5b, 0x34, 0xba, 0x7d, 0x3c, 0xeb, 0x0a, + 0xc7, 0x63, 0x65, 0x0b, 0x52, 0x34, 0x39, 0x75, 0xca, 0xf3, 0x74, 0xdf, 0xaf, 0x99, 0xe8, 0xca, + 0x5c, 0x0a, 0xda, 0x83, 0xa2, 0x46, 0x20, 0xa4, 0x72, 0x62, 0xfd, 0x94, 0xdb, 0x54, 0xee, 0xf2, + 0x14, 0xb9, 0xc2, 0xd1, 0x8a, 0xb2, 0x59, 0x30, 0xb3, 0x9e, 0x4b, 0x1b, 0x32, 0x5d, 0xb5, 0xaf, + 0x1b, 0x3a, 0x76, 0xca, 0x0b, 0x54, 0xce, 0x27, 0x97, 0x5a, 0xf5, 0x68, 0x6f, 0x5d, 0x04, 0x57, + 0x21, 0xc4, 0x33, 0x6e, 0x0a, 0x38, 0x23, 0x87, 0x78, 0x67, 0xdc, 0xb8, 0x45, 0x6f, 0x3d, 0xd4, + 0x67, 0xa7, 0xc6, 0xcd, 0xbf, 0x34, 0xf4, 0x00, 0xe0, 0xb5, 0x8e, 0xdf, 0x28, 0xaf, 0x86, 0xd8, + 0x3e, 0x2b, 0x97, 0x83, 0x65, 0x12, 0x81, 0x7f, 0x49, 0xc0, 0xe8, 0x53, 0xc8, 0x6a, 0x78, 0x80, + 0x4d, 0xcd, 0x69, 0x9b, 0xe5, 0xbb, 0x34, 0x29, 0xbd, 0x75, 0x71, 0xbe, 0x9c, 0xad, 0x0b, 0x20, + 0xf7, 0xa5, 0x3e, 0x15, 0xfa, 0x0a, 0xf2, 0xec, 0x03, 0x6b, 0x6d, 0x73, 0xf3, 0xac, 0xbc, 0x48, + 0x17, 0xfd, 0x68, 0xc6, 0x43, 0xf1, 0x9b, 0x10, 0x5e, 0xdf, 0xb6, 0x1e, 0x90, 0x26, 0x87, 0x64, + 0xa3, 0xdf, 0x87, 0xbc, 0xd0, 0xe6, 0x1d, 0xeb, 0xd8, 0x29, 0xbf, 0x73, 0x69, 0x53, 0x75, 0x74, + 0xac, 0x7d, 0x9f, 0x55, 0xf8, 0xaa, 0xa0, 0x34, 0xf4, 0x73, 0x28, 0x78, 0x17, 0x44, 0xd6, 0xc0, + 0x75, 0xca, 0xf7, 0xa8, 0x31, 0x3e, 0x9e, 0x55, 0x55, 0x39, 0x6f, 0x7b, 0x40, 0xfb, 0xcd, 0x81, + 0x2f, 0x74, 0x1f, 0xb2, 0x9a, 0x6d, 0x0d, 0x58, 0xcc, 0x78, 0x77, 0x25, 0xba, 0x1a, 0xf7, 0x2a, + 0x61, 0xdb, 0x1a, 0xd0, 0x60, 0xa0, 0x40, 0xd1, 0xc6, 0x03, 0x43, 0xed, 0xe0, 0x3e, 0x89, 0x66, + 0x56, 0xb7, 0xbc, 0x44, 0x47, 0xdf, 0x98, 0x79, 0x23, 0x3d, 0x66, 0xa1, 0x98, 0x01, 0x79, 0xed, + 0x2e, 0x7a, 0x06, 0xa0, 0x0e, 0x35, 0xdd, 0x55, 0xfa, 0x96, 0x86, 0xcb, 0xcb, 0x97, 0xde, 0x5a, + 0x8e, 0x0a, 0xaf, 0x12, 0xc6, 0x7d, 0x4b, 0xc3, 0xde, 0x0d, 0x85, 0x00, 0xa0, 0x4f, 0x21, 0x47, + 0x97, 0xf6, 0x95, 0x75, 0x4c, 0x74, 0x73, 0x85, 0x2e, 0x6e, 0x8e, 0x9f, 0x65, 0xb6, 0x6e, 0x5b, + 0x83, 0x1d, 0xeb, 0x98, 0x6a, 0x0c, 0xff, 0xa9, 0x21, 0x07, 0xf2, 0xbd, 0x8e, 0xe2, 0xbb, 0xcf, + 0xfb, 0xf4, 0x14, 0x3f, 0x9f, 0x71, 0x2e, 0x4f, 0x6b, 0x13, 0x1c, 0xea, 0x2d, 0x11, 0x07, 0x9e, + 0xd6, 0x04, 0xcc, 0x91, 0x73, 0xbd, 0x8e, 0xf7, 0x41, 0xea, 0x0e, 0xd6, 0x6e, 0xe1, 0x06, 0x50, + 0x09, 0xd6, 0x1d, 0x0c, 0xc3, 0x4c, 0xa0, 0x05, 0xbc, 0x2f, 0xa3, 0xa8, 0x8e, 0x62, 0x75, 0xd9, + 0x99, 0x3d, 0x98, 0x3d, 0xce, 0x17, 0x19, 0x77, 0xd5, 0x69, 0x77, 0xe9, 0xc1, 0x76, 0x20, 0x6f, + 0x0d, 0xdd, 0x63, 0x6b, 0x68, 0x6a, 0x4a, 0xf7, 0xa5, 0x53, 0x7e, 0x8f, 0xae, 0xf6, 0x4a, 0x35, + 0xb4, 0xb7, 0xba, 0x36, 0x17, 0xb4, 0xb5, 0xeb, 0xc8, 0x39, 0x21, 0x75, 0xeb, 0xa5, 0x83, 0xfe, + 0x08, 0x72, 0xba, 0xe9, 0x8f, 0xf1, 0xf0, 0xea, 0x63, 0x20, 0x51, 0x13, 0x34, 0x4d, 0x6f, 0x08, + 0xe0, 0x32, 0xc9, 0x08, 0x1f, 0x41, 0xd1, 0xea, 0x76, 0x0d, 0xdd, 0xc4, 0x8a, 0x8d, 0x55, 0xc7, + 0x32, 0xcb, 0xef, 0x07, 0x76, 0xb0, 0xc0, 0x71, 0x32, 0x45, 0xa1, 0x0a, 0x64, 0x5d, 0xdc, 0x1f, + 0x58, 0xb6, 0x6a, 0x9f, 0x95, 0x3f, 0x08, 0x5e, 0xec, 0x78, 0x60, 0x74, 0x0c, 0x8b, 0x43, 0x13, + 0x9f, 0x0e, 0x2c, 0x07, 0x6b, 0xca, 0x58, 0x86, 0xb9, 0x4a, 0x7d, 0xdc, 0x43, 0x3e, 0xa9, 0x3b, + 0xcf, 0x04, 0xe5, 0xc4, 0x54, 0xf3, 0xce, 0x70, 0x22, 0x5a, 0x5b, 0xfc, 0x75, 0x14, 0xe6, 0xc6, + 0x62, 0x24, 0xfa, 0x43, 0x48, 0x9b, 0x96, 0x16, 0xb8, 0x46, 0x6b, 0xf0, 0x61, 0x52, 0x2d, 0x4b, + 0x63, 0xb7, 0x68, 0x8f, 0x7b, 0xba, 0x7b, 0x32, 0x3c, 0x5e, 0xeb, 0x58, 0xfd, 0x75, 0x6f, 0x13, + 0xb5, 0x63, 0xff, 0xf7, 0xfa, 0xe0, 0x65, 0x6f, 0x9d, 0xfe, 0x1a, 0x1c, 0xaf, 0x31, 0x36, 0x39, + 0x45, 0xa4, 0x36, 0x35, 0xf4, 0x09, 0x94, 0xf0, 0xe9, 0x40, 0xb7, 0x03, 0x79, 0x62, 0x2c, 0x60, + 0xf3, 0x45, 0x1f, 0x49, 0x14, 0x84, 0x5f, 0x74, 0xfc, 0x5d, 0x0c, 0x4a, 0x23, 0x11, 0x8a, 0x24, + 0xc6, 0xb4, 0x46, 0x0e, 0x25, 0xc6, 0x04, 0x72, 0x49, 0xba, 0x1e, 0x7c, 0x90, 0x11, 0xbf, 0xe9, + 0x83, 0x8c, 0xf0, 0x05, 0x42, 0xf2, 0x0a, 0x17, 0x08, 0x3f, 0x85, 0x05, 0xdd, 0x51, 0x4c, 0xcb, + 0x14, 0x7d, 0x1e, 0xaf, 0x3c, 0x0c, 0x5e, 0xfb, 0xdf, 0xd2, 0x9d, 0x96, 0x65, 0xb2, 0x0e, 0x8f, + 0xb7, 0x6a, 0xff, 0x85, 0x40, 0x7a, 0xfc, 0x85, 0x80, 0xd7, 0xc7, 0x49, 0x48, 0xc9, 0xc5, 0x6f, + 0xa2, 0x90, 0x0d, 0x3e, 0x33, 0x8b, 0x85, 0x5b, 0x17, 0x63, 0xc5, 0xc2, 0x35, 0xaf, 0x43, 0xc3, + 0xbb, 0x10, 0x9f, 0x7d, 0x17, 0xf8, 0xd1, 0xfe, 0x31, 0xe4, 0x02, 0xa1, 0x67, 0xb4, 0x24, 0x8f, + 0x5e, 0xa3, 0x24, 0x7f, 0x0f, 0x52, 0xdc, 0xdf, 0x32, 0xc5, 0x2a, 0x70, 0xee, 0x24, 0xf3, 0xb5, + 0xc9, 0xaf, 0x88, 0x9f, 0xe5, 0xa3, 0xff, 0x4b, 0x1c, 0xf2, 0xc1, 0xd0, 0x44, 0x8c, 0x53, 0x37, + 0x3b, 0x36, 0x8d, 0x0b, 0x74, 0xf4, 0xb8, 0x77, 0xeb, 0x2a, 0xc0, 0x24, 0x60, 0xf5, 0x75, 0x53, + 0xa1, 0x37, 0x7d, 0x21, 0xe5, 0xcd, 0xf4, 0x75, 0xf3, 0x39, 0x81, 0x52, 0x12, 0xf5, 0x94, 0x93, + 0xc4, 0x43, 0x24, 0xea, 0x29, 0x23, 0x59, 0xa4, 0x39, 0x9f, 0xed, 0xd2, 0xc2, 0x2c, 0x1e, 0xc8, + 0xe2, 0x6c, 0x17, 0x2d, 0x41, 0xfa, 0xb5, 0x6e, 0xbb, 0x43, 0xd5, 0xa0, 0x35, 0x98, 0x38, 0x66, + 0x01, 0x44, 0x26, 0x14, 0xfd, 0x60, 0xfc, 0xc6, 0xc4, 0x36, 0x55, 0x9c, 0xdc, 0x46, 0xf5, 0x1a, + 0xd1, 0xd8, 0xff, 0x20, 0x82, 0x84, 0xcb, 0x72, 0x82, 0xc0, 0xc5, 0xbf, 0x8c, 0x42, 0x21, 0x44, + 0x86, 0x9a, 0x50, 0xa2, 0x03, 0x8f, 0xf5, 0xc5, 0xee, 0x7b, 0x0f, 0xc6, 0x08, 0x7a, 0x62, 0x5d, + 0x55, 0xb0, 0x02, 0x28, 0x0d, 0x7d, 0x01, 0x45, 0x26, 0xca, 0xbb, 0xbf, 0x0f, 0xab, 0x5f, 0x9e, + 0x4a, 0x0a, 0x5f, 0xe2, 0xe7, 0x2d, 0x1f, 0xa6, 0x05, 0xaf, 0x26, 0x17, 0x4d, 0xc8, 0x05, 0xa2, + 0xfd, 0x0c, 0x7a, 0xff, 0x13, 0x48, 0x78, 0x5e, 0x68, 0xc6, 0x28, 0x46, 0x19, 0xf8, 0x78, 0x5f, + 0x47, 0x61, 0x7e, 0x52, 0xd4, 0x0d, 0xd9, 0x13, 0x53, 0xa4, 0x99, 0xec, 0xe9, 0x41, 0x30, 0x1b, + 0x62, 0xca, 0x25, 0xae, 0xcb, 0xfc, 0x7c, 0xe8, 0x7d, 0x4f, 0xc5, 0x99, 0x6e, 0x95, 0x42, 0x2a, + 0x4e, 0xaa, 0x9c, 0x80, 0x92, 0x57, 0x1e, 0x8b, 0x36, 0x0c, 0x40, 0xea, 0xe0, 0xd9, 0xe6, 0x5e, + 0xb3, 0x36, 0xb1, 0x85, 0x82, 0x72, 0x90, 0x6e, 0x6f, 0x6d, 0xed, 0x35, 0x5b, 0x0d, 0x29, 0x5e, + 0x59, 0x85, 0xac, 0x97, 0xd8, 0xa0, 0x3c, 0x64, 0xea, 0xcd, 0xc3, 0xea, 0xe6, 0x5e, 0xa3, 0x2e, + 0x45, 0x50, 0x01, 0xb2, 0x72, 0xa3, 0x5a, 0xa7, 0x8d, 0x1a, 0x29, 0xfa, 0x59, 0xe6, 0x57, 0x5f, + 0x2f, 0x47, 0xb9, 0xe3, 0x49, 0x49, 0xe9, 0x9d, 0x44, 0x06, 0x49, 0xb7, 0x2a, 0xff, 0x13, 0x03, + 0x54, 0x57, 0x5d, 0x95, 0xe8, 0xdf, 0x15, 0xda, 0x19, 0xb1, 0x4b, 0x4e, 0x2a, 0x5c, 0xa2, 0xc6, + 0x6f, 0x54, 0xa2, 0x4e, 0x6c, 0x58, 0x24, 0x6e, 0xd2, 0xb0, 0xb8, 0x56, 0xdf, 0x64, 0xbc, 0xbc, + 0x4a, 0x5d, 0xbf, 0xbc, 0xf2, 0x4f, 0xa2, 0x72, 0x91, 0x82, 0xe2, 0xd1, 0xd9, 0x20, 0xb8, 0xf3, + 0xd7, 0xea, 0x7c, 0x4d, 0xea, 0x6f, 0xc5, 0xae, 0xde, 0xdf, 0xba, 0xe4, 0x71, 0x32, 0x3b, 0xf6, + 0xc4, 0x25, 0xc7, 0x5e, 0x87, 0x04, 0x7d, 0xd1, 0x95, 0xa4, 0x01, 0x7b, 0xda, 0x81, 0x87, 0x57, + 0xbb, 0x16, 0x78, 0xd4, 0x45, 0xb9, 0xd1, 0x2f, 0x20, 0x8f, 0xcd, 0x61, 0x5f, 0xe9, 0xe3, 0xfe, + 0x31, 0xb6, 0xc5, 0x3e, 0x3f, 0x9a, 0x4d, 0x5a, 0xc3, 0x1c, 0xf6, 0xf7, 0x29, 0xa3, 0x48, 0x8a, + 0xb1, 0x07, 0x71, 0xd0, 0x23, 0x48, 0xaa, 0x86, 0xae, 0x3a, 0xbc, 0xe1, 0x75, 0xd9, 0x5b, 0x41, + 0x46, 0x88, 0x7e, 0x17, 0x0a, 0xaa, 0x6d, 0xab, 0x67, 0xfc, 0xcd, 0x9b, 0x46, 0xdb, 0xc2, 0x5c, + 0x63, 0x48, 0x2a, 0x5b, 0x25, 0x48, 0xfa, 0xcc, 0x4d, 0x6c, 0x44, 0x4e, 0xf5, 0x40, 0xa1, 0x16, + 0x5d, 0xf6, 0x66, 0x2d, 0x3a, 0xb8, 0x89, 0xc6, 0x8f, 0x2b, 0x6f, 0xee, 0xfa, 0xca, 0xbb, 0xf8, + 0xab, 0x28, 0x80, 0xbf, 0xcf, 0xe8, 0x27, 0x70, 0x67, 0x70, 0x72, 0xe6, 0xd0, 0x27, 0x86, 0x36, + 0x1e, 0xd8, 0xd8, 0xc1, 0x26, 0xf3, 0xac, 0x54, 0x79, 0xf3, 0xf2, 0x82, 0x40, 0xcb, 0x21, 0x2c, + 0xfa, 0x1c, 0x16, 0xc4, 0xd3, 0xc4, 0x11, 0xbe, 0x60, 0x3e, 0x78, 0x9b, 0xd3, 0x84, 0x99, 0xb9, + 0xc3, 0x7c, 0x07, 0x12, 0x44, 0x7f, 0x88, 0x6b, 0x6c, 0xb4, 0x9e, 0xed, 0x4b, 0x11, 0x94, 0x85, + 0x64, 0x75, 0xaf, 0x59, 0x3d, 0x0c, 0xba, 0xbb, 0xca, 0xdf, 0xc4, 0x41, 0x62, 0x1a, 0x7e, 0x53, + 0x33, 0x9b, 0x9e, 0xb1, 0xfe, 0x70, 0x93, 0x37, 0xec, 0x15, 0x13, 0x6f, 0xdf, 0x2b, 0x26, 0xdf, + 0x92, 0x57, 0x4c, 0xdd, 0xc0, 0x2b, 0xa6, 0xdf, 0x8a, 0x57, 0xfc, 0x26, 0x06, 0x10, 0x38, 0xaa, + 0x9f, 0x05, 0xff, 0xf5, 0x62, 0x7a, 0xf3, 0x75, 0x24, 0x79, 0xda, 0x8e, 0x88, 0x7f, 0xcc, 0x78, + 0x0a, 0x19, 0x8d, 0x47, 0x38, 0x9e, 0x3d, 0x4c, 0xed, 0x72, 0x8e, 0x05, 0xc2, 0xed, 0x88, 0xec, + 0x31, 0xa3, 0xcf, 0x43, 0x6f, 0x8d, 0x1f, 0xce, 0xe4, 0x93, 0xb6, 0xc5, 0x53, 0x8a, 0x2a, 0xa4, + 0x98, 0x6f, 0xe6, 0x67, 0x3f, 0xf5, 0xc9, 0xeb, 0x88, 0xa6, 0x6e, 0x47, 0x64, 0xce, 0xc8, 0xef, + 0x4e, 0xd2, 0x90, 0x1c, 0x9a, 0xba, 0x65, 0x7e, 0x28, 0x07, 0x2f, 0xf1, 0x45, 0xf9, 0x43, 0xa2, + 0x3e, 0xfd, 0xad, 0xba, 0x58, 0x63, 0xb7, 0x38, 0xcf, 0xcc, 0xd7, 0x1e, 0x20, 0x8a, 0x8a, 0x00, + 0x1c, 0xaf, 0x9b, 0x3d, 0x29, 0x46, 0x73, 0x06, 0xdb, 0x1a, 0x0c, 0xc8, 0x57, 0xfc, 0xc3, 0x75, + 0x90, 0x46, 0xdf, 0xdc, 0x06, 0xee, 0x73, 0xe6, 0xa0, 0xb0, 0xff, 0xbc, 0x56, 0x3b, 0x6a, 0xee, + 0x37, 0x0e, 0x8f, 0xaa, 0xfb, 0x07, 0x52, 0x74, 0xf3, 0x47, 0xdf, 0xfe, 0xfb, 0x52, 0xe4, 0xdb, + 0x8b, 0xa5, 0xe8, 0x6f, 0x2e, 0x96, 0xa2, 0xbf, 0xbd, 0x58, 0x8a, 0xfe, 0xdb, 0xc5, 0x52, 0xf4, + 0xcf, 0xbe, 0x5b, 0x8a, 0xfc, 0xe6, 0xbb, 0xa5, 0xc8, 0x6f, 0xbf, 0x5b, 0x8a, 0xfc, 0x5e, 0x9a, + 0xaf, 0xec, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x8e, 0x1e, 0x60, 0x7f, 0x0f, 0x34, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index c4596ec442c6..c750f9b99827 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -139,8 +139,22 @@ message ColumnDescriptor { (gogoproto.customname) = "LogicalColumnID", (gogoproto.casttype) = "ColumnID"]; // Used to indicate column is used and dropped for ALTER COLUMN TYPE mutation. optional bool alter_column_type_in_progress = 14 [(gogoproto.nullable) = false]; + + // SystemColumnKind represents what kind of system column this column + // descriptor represents, if any. + optional SystemColumnKind system_column_kind = 15 [(gogoproto.nullable) = false]; } - + +// SystemColumnKind is an enum representing the different kind of system +// columns that can be synthesized by the execution engine. +enum SystemColumnKind { + // Default value, unused. + NONE = 0; + // A system column containing the value of the MVCC timestamp associated + // with the kv's corresponding to the row. + MVCCTIMESTAMP = 1; +} + // ColumnFamilyDescriptor is set of columns stored together in one kv entry. // For more information, look at `docs/tech-notes/encoding.md#value-encoding`. message ColumnFamilyDescriptor { diff --git a/pkg/sql/sqlbase/system_columns.go b/pkg/sql/sqlbase/system_columns.go new file mode 100644 index 000000000000..78cfda9b5ca4 --- /dev/null +++ b/pkg/sql/sqlbase/system_columns.go @@ -0,0 +1,137 @@ +// Copyright 2020 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. + +package sqlbase + +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" +) + +// Similar to Postgres, we also expose hidden system columns on tables. +// These system columns are not explicitly part of the TableDescriptor, +// and instead are constructs that are manipulated purely during planning. +// As of now, these system columns are able to be generated from the set +// of KV's that make up a row, and are produced by the row and cfetcher. +// Each system column is given a kind, and associated with a particular +// column ID that is counting down from math.MaxUint32. This is done so that +// each system column ID won't conflict with existing column ID's and also +// will be stable across all changes to the table. +// * MVCC Timestamp: contains a representation of the row's HLC timestamp. + +// MVCCTimestampColumnName is the name of the MVCC timestamp system column. +const MVCCTimestampColumnName = "crdb_internal_mvcc_timestamp" + +// MVCCTimestampColumnType is the type of the MVCC timestamp system column. +var MVCCTimestampColumnType = types.Decimal + +// MVCCTimestampColumnID is the ColumnID of the MVCC timesatmp column. Future +// system columns will have ID's that decrement from this value. +const MVCCTimestampColumnID = math.MaxUint32 + +// NewMVCCTimestampColumnDesc creates a column descriptor for the MVCC timestamp +// system column. +func NewMVCCTimestampColumnDesc() *ColumnDescriptor { + return &ColumnDescriptor{ + Name: MVCCTimestampColumnName, + Type: MVCCTimestampColumnType, + Hidden: true, + Nullable: true, + SystemColumnKind: SystemColumnKind_MVCCTIMESTAMP, + ID: MVCCTimestampColumnID, + } +} + +// IsColIDSystemColumn returns whether a column ID refers to a system column. +func IsColIDSystemColumn(colID ColumnID) bool { + switch colID { + case MVCCTimestampColumnID: + return true + default: + return false + } +} + +// GetSystemColumnDescriptorFromID returns a column descriptor corresponding +// to the system column referred to by the input column ID. +func GetSystemColumnDescriptorFromID(colID ColumnID) (*ColumnDescriptor, error) { + switch colID { + case MVCCTimestampColumnID: + return NewMVCCTimestampColumnDesc(), nil + default: + return nil, errors.AssertionFailedf("unsupported system column ID %d", colID) + } +} + +// GetSystemColumnKindFromColumnID returns the kind of system column that colID +// refers to. +func GetSystemColumnKindFromColumnID(colID ColumnID) SystemColumnKind { + switch colID { + case MVCCTimestampColumnID: + return SystemColumnKind_MVCCTIMESTAMP + default: + return SystemColumnKind_NONE + } +} + +// GetSystemColumnIDByKind returns the column ID of the desired system column. +func GetSystemColumnIDByKind(kind SystemColumnKind) (ColumnID, error) { + switch kind { + case SystemColumnKind_MVCCTIMESTAMP: + return MVCCTimestampColumnID, nil + default: + return 0, errors.Newf("invalid system column kind %s", kind.String()) + } +} + +// GetSystemColumnTypeForKind returns the types.T of the input system column. +func GetSystemColumnTypeForKind(kind SystemColumnKind) *types.T { + switch kind { + case SystemColumnKind_MVCCTIMESTAMP: + return MVCCTimestampColumnType + default: + return nil + } +} + +// IsSystemColumnName returns whether or not a name is a reserved system +// column name. +func IsSystemColumnName(name string) bool { + switch name { + case MVCCTimestampColumnName: + return true + default: + return false + } +} + +// GetSystemColumnTypesAndDescriptors is a utility method to construct a set of +// types and column descriptors from an input list of system column kinds. +func GetSystemColumnTypesAndDescriptors( + desc *TableDescriptor, kinds []SystemColumnKind, +) ([]*types.T, []ColumnDescriptor, error) { + resTypes := make([]*types.T, len(kinds)) + resDescs := make([]ColumnDescriptor, len(kinds)) + for i, k := range kinds { + resTypes[i] = GetSystemColumnTypeForKind(k) + colID, err := GetSystemColumnIDByKind(k) + if err != nil { + return nil, nil, err + } + colDesc, err := GetSystemColumnDescriptorFromID(colID) + if err != nil { + return nil, nil, err + } + resDescs[i] = *colDesc + } + return resTypes, resDescs, nil +} diff --git a/pkg/sql/sqlbase/validate_test.go b/pkg/sql/sqlbase/validate_test.go index 043e5bebc3e9..a77dfe89b729 100644 --- a/pkg/sql/sqlbase/validate_test.go +++ b/pkg/sql/sqlbase/validate_test.go @@ -174,6 +174,7 @@ var validationMap = []struct { status: todoIAmKnowinglyAddingTechDebt, reason: "initial import: TODO(features): add validation"}, "AlterColumnTypeInProgress": {status: thisFieldReferencesNoObjects}, + "SystemColumnKind": {status: thisFieldReferencesNoObjects}, }, }, { diff --git a/pkg/sql/testdata/explain_tree b/pkg/sql/testdata/explain_tree index fcdf4558005a..46b9e2de3f11 100644 --- a/pkg/sql/testdata/explain_tree +++ b/pkg/sql/testdata/explain_tree @@ -58,9 +58,9 @@ SELECT cid, sum(value) FROM t.orders WHERE date > '2015-01-01' GROUP BY cid ORDE render (cid int, sum decimal) │ render 0 (@2)[int] │ render 1 (@3)[decimal] - └── sort (column6 decimal, cid int, sum decimal) +column6 - │ order +column6 - └── render (column6 decimal, cid int, sum decimal) + └── sort (column7 decimal, cid int, sum decimal) +column7 + │ order +column7 + └── render (column7 decimal, cid int, sum decimal) │ render 0 ((1)[decimal] - (@2)[decimal])[decimal] │ render 1 (@1)[int] │ render 2 (@2)[decimal] @@ -89,7 +89,7 @@ children: - name: sort attrs: - key: order - value: +column6 + value: +column7 children: - name: render attrs: