diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 345dc6933a0b..007736dc8d83 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -115,6 +115,7 @@ type TableDescriptor interface { IndexSpan(codec keys.SQLCodec, id descpb.IndexID) roachpb.Span FindIndexByID(id descpb.IndexID) (*descpb.IndexDescriptor, error) FindIndexByName(name string) (_ *descpb.IndexDescriptor, dropped bool, _ error) + FindIndexByIndexIdx(idx int) (*descpb.IndexDescriptor, bool, error) FindIndexesWithPartition(name string) []*descpb.IndexDescriptor GetIndexMutationCapabilities(id descpb.IndexID) (isMutation, isWriteOnly bool) KeysPerRow(id descpb.IndexID) (int, error) @@ -125,6 +126,7 @@ type TableDescriptor interface { PrimaryKeyString() string GetPublicColumns() []descpb.ColumnDescriptor + GetReadableColumns() []descpb.ColumnDescriptor ForeachPublicColumn(f func(col *descpb.ColumnDescriptor) error) error ForeachNonDropColumn(f func(col *descpb.ColumnDescriptor) error) error NamesForColumnIDs(ids descpb.ColumnIDs) ([]string, error) @@ -172,6 +174,8 @@ type TableDescriptor interface { ForeachInboundFK(f func(fk *descpb.ForeignKeyConstraint) error) error FindActiveColumnByName(s string) (*descpb.ColumnDescriptor, error) WritableColumns() []descpb.ColumnDescriptor + ColumnTypes() []*types.T + ColumnTypesWithMutations(mutations bool) []*types.T } // TypeDescriptor will eventually be called typedesc.Descriptor. diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index d7c4c7a4dcff..db49b35cb414 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -536,7 +537,7 @@ func (tc *Collection) getUserDefinedSchemaVersion( // the database which doesn't reflect the changes to the schema. But this // isn't a problem for correctness; it can only happen on other sessions // before the schema change has returned results. - desc, err := tc.getDescriptorVersionByID(ctx, txn, schemaInfo.ID, flags.CommonLookupFlags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, txn.ReadTimestamp(), schemaInfo.ID, flags.CommonLookupFlags, true /* setTxnDeadline */) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { return nil, sqlerrors.NewUndefinedSchemaError(schemaName) @@ -853,7 +854,7 @@ func (tc *Collection) GetDatabaseVersionByID( return tc.deprecatedDatabaseCache().GetDatabaseDescByID(ctx, txn, dbID) } - desc, err := tc.getDescriptorVersionByID(ctx, txn, dbID, flags.CommonLookupFlags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, txn.ReadTimestamp(), dbID, flags.CommonLookupFlags, true /* setTxnDeadline */) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { return nil, sqlerrors.NewUndefinedDatabaseError(fmt.Sprintf("[%d]", dbID)) @@ -867,11 +868,34 @@ func (tc *Collection) GetDatabaseVersionByID( return db, nil } -// GetTableVersionByID is a by-ID variant of GetTableVersion (i.e. uses same cache). -func (tc *Collection) GetTableVersionByID( - ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, +func (tc *Collection) GetTableVersionByIDWithMinVersion( + ctx context.Context, + minTimestamp hlc.Timestamp, + txn *kv.Txn, + tableID descpb.ID, + version descpb.DescriptorVersion, + flags tree.ObjectLookupFlags, +) (*tabledesc.Immutable, error) { + desc, err := tc.getTableVersionByIDImpl(ctx, txn, minTimestamp, tableID, flags) + if err != nil { + return nil, err + } + if desc.GetVersion() != version { + if err := tc.leaseMgr.AcquireFreshestFromStore(ctx, desc.GetID()); err != nil { + return nil, err + } + } + return tc.getTableVersionByIDImpl(ctx, txn, minTimestamp, tableID, flags) +} + +func (tc *Collection) getTableVersionByIDImpl( + ctx context.Context, + txn *kv.Txn, + readTimestamp hlc.Timestamp, + tableID descpb.ID, + flags tree.ObjectLookupFlags, ) (*tabledesc.Immutable, error) { - desc, err := tc.getDescriptorVersionByID(ctx, txn, tableID, flags.CommonLookupFlags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, readTimestamp, tableID, flags.CommonLookupFlags, txn != nil) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { return nil, sqlerrors.NewUndefinedRelationError( @@ -891,8 +915,20 @@ func (tc *Collection) GetTableVersionByID( return hydrated.(*tabledesc.Immutable), nil } +// GetTableVersionByID is a by-ID variant of GetTableVersion (i.e. uses same cache). +func (tc *Collection) GetTableVersionByID( + ctx context.Context, txn *kv.Txn, tableID descpb.ID, flags tree.ObjectLookupFlags, +) (*tabledesc.Immutable, error) { + return tc.getTableVersionByIDImpl(ctx, txn, txn.ReadTimestamp(), tableID, flags) +} + func (tc *Collection) getDescriptorVersionByID( - ctx context.Context, txn *kv.Txn, id descpb.ID, flags tree.CommonLookupFlags, setTxnDeadline bool, + ctx context.Context, + txn *kv.Txn, + readTimestamp hlc.Timestamp, + id descpb.ID, + flags tree.CommonLookupFlags, + setTxnDeadline bool, ) (catalog.Descriptor, error) { if flags.AvoidCached || lease.TestingTableLeasesAreDisabled() { desc, err := catalogkv.GetDescriptorByID(ctx, txn, tc.codec(), id, catalogkv.Immutable, @@ -926,7 +962,6 @@ func (tc *Collection) getDescriptorVersionByID( return desc, nil } - readTimestamp := txn.ReadTimestamp() desc, expiration, err := tc.leaseMgr.Acquire(ctx, readTimestamp, id) if err != nil { return nil, err @@ -939,7 +974,7 @@ func (tc *Collection) getDescriptorVersionByID( tc.leasedDescriptors.add(desc) log.VEventf(ctx, 2, "added descriptor %q to collection", desc.GetName()) - if setTxnDeadline { + if setTxnDeadline && txn != nil { // If the descriptor we just acquired expires before the txn's deadline, // reduce the deadline. We use ReadTimestamp() that doesn't return the commit // timestamp, so we need to set a deadline on the transaction to prevent it @@ -1018,7 +1053,7 @@ func (tc *Collection) ResolveSchemaByID( // Otherwise, fall back to looking up the descriptor with the desired ID. desc, err := tc.getDescriptorVersionByID( - ctx, txn, schemaID, tree.CommonLookupFlags{Required: true}, true /* setTxnDeadline */) + ctx, txn, txn.ReadTimestamp(), schemaID, tree.CommonLookupFlags{Required: true}, true /* setTxnDeadline */) if err != nil { return catalog.ResolvedSchema{}, err } @@ -1319,7 +1354,7 @@ func (tc *Collection) GetTypeVersion( func (tc *Collection) GetTypeVersionByID( ctx context.Context, txn *kv.Txn, typeID descpb.ID, flags tree.ObjectLookupFlags, ) (*typedesc.Immutable, error) { - desc, err := tc.getDescriptorVersionByID(ctx, txn, typeID, flags.CommonLookupFlags, true /* setTxnDeadline */) + desc, err := tc.getDescriptorVersionByID(ctx, txn, txn.ReadTimestamp(), typeID, flags.CommonLookupFlags, true /* setTxnDeadline */) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { return nil, pgerror.Newf( @@ -1701,6 +1736,7 @@ func (dt *DistSQLTypeResolver) GetTypeDescriptor( desc, err := dt.descriptors.getDescriptorVersionByID( ctx, dt.txn, + dt.txn.ReadTimestamp(), id, tree.CommonLookupFlags{Required: true}, false, /* setTxnDeadline */ diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index 50219eb4321a..75f84ebf5cdf 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -151,11 +151,12 @@ func MakeImmutable(tbl descpb.TableDescriptor) Immutable { } // Iterate through all mutation columns. - for _, c := range publicAndNonPublicCols[len(tbl.Columns):] { + for i := range publicAndNonPublicCols[len(tbl.Columns):] { // Mutation column may need to be fetched, but may not be completely backfilled // and have be null values (even though they may be configured as NOT NULL). + c := &publicAndNonPublicCols[i] c.Nullable = true - readableCols = append(readableCols, c) + readableCols = append(readableCols, *c) } } @@ -2472,7 +2473,7 @@ func (desc *Immutable) FindColumnMutationByName(name tree.Name) *descpb.Descript } // ColumnIdxMap returns a map from Column ID to the ordinal position of that -// column. +// column. It must not be modified as it is shared. func (desc *Immutable) ColumnIdxMap() map[descpb.ColumnID]int { return desc.ColumnIdxMapWithMutations(false) } diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index adb5b89dd67a..a7b68407e0e1 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -49,6 +49,7 @@ type Immutable struct { // are all set to nullable while column backfilling is still in // progress, as mutation columns may have NULL values. ReadableColumns []descpb.ColumnDescriptor + // columnIdxMap map[descpb.ColumnID]int // columnsWithUDTs is a set of indexes into publicAndNonPublicCols containing // indexes of columns that contain user defined types. @@ -123,3 +124,7 @@ func (desc *Mutable) IsUncommittedVersion() bool { func (desc *Mutable) SetDrainingNames(names []descpb.NameInfo) { desc.DrainingNames = names } + +func (desc *Immutable) GetReadableColumns() []descpb.ColumnDescriptor { + return desc.ReadableColumns +} diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index b343eb51caed..f07d185685a0 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -137,30 +138,58 @@ func NewColBatchScan( // indicates that we're probably doing this wrong. Instead we should be // just seting the ID and Version in the spec or something like that and // retrieving the hydrated Immutable from cache. - table := tabledesc.NewImmutable(spec.Table) + var table catalog.TableDescriptor + if spec.TableIsUncommittedVersion { + var err error + table, err = flowCtx.TypeResolverFactory.Descriptors.GetTableVersionByIDWithMinVersion( + flowCtx.EvalCtx.Ctx(), + spec.Table.ModificationTime, + flowCtx.EvalCtx.Txn, + spec.Table.ID, spec.Table.Version, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return nil, err + } + } else { + table = tabledesc.NewImmutable(spec.Table) + } + typs := table.ColumnTypesWithMutations(returnMutations) columnIdxMap := table.ColumnIdxMapWithMutations(returnMutations) + // Add all requested system columns to the output. - sysColTypes, sysColDescs, err := colinfo.GetSystemColumnTypesAndDescriptors(spec.SystemColumns) - if err != nil { - return nil, err - } - typs = append(typs, sysColTypes...) - for i := range sysColDescs { - columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + var sysColDescs []descpb.ColumnDescriptor + if len(spec.SystemColumns) > 0 { + columnIdxMapCpy := make(map[descpb.ColumnID]int, len(columnIdxMap)+len(spec.SystemColumns)) + for id, idx := range columnIdxMap { + columnIdxMapCpy[id] = idx + } + columnIdxMap = columnIdxMapCpy + var sysColTypes []*types.T + var err error + sysColTypes, sysColDescs, err = colinfo.GetSystemColumnTypesAndDescriptors(spec.SystemColumns) + if err != nil { + return nil, err + } + typs = append(typs, sysColTypes...) + for i := range sysColDescs { + columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + } } semaCtx := tree.MakeSemaContext() 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 - // processors, so we need to do the hydration ourselves. - resolver := flowCtx.TypeResolverFactory.NewTypeResolver(evalCtx.Txn) - semaCtx.TypeResolver = resolver - if err := resolver.HydrateTypeSlice(evalCtx.Context, typs); err != nil { - return nil, err + if spec.TableIsUncommittedVersion { + // 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 + // processors, so we need to do the hydration ourselves. + resolver := flowCtx.TypeResolverFactory.NewTypeResolver(evalCtx.Txn) + semaCtx.TypeResolver = resolver + if err := resolver.HydrateTypeSlice(evalCtx.Context, typs); err != nil { + return nil, err + } } + helper := execinfra.ProcOutputHelper{} if err := helper.Init( post, @@ -227,7 +256,7 @@ func initCRowFetcher( cols := immutDesc.Columns if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { - cols = immutDesc.ReadableColumns + cols = immutDesc.GetReadableColumns() } // Add on any requested system columns. We slice cols to avoid modifying // the underlying table descriptor. diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 56c1ff4370c4..4d392d5a6beb 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -941,12 +941,13 @@ func initTableReaderSpec( ) (*execinfrapb.TableReaderSpec, execinfrapb.PostProcessSpec, error) { s := physicalplan.NewTableReaderSpec() *s = execinfrapb.TableReaderSpec{ - Table: *n.desc.TableDesc(), - Reverse: n.reverse, - IsCheck: n.isCheck, - Visibility: n.colCfg.visibility, - LockingStrength: n.lockingStrength, - LockingWaitPolicy: n.lockingWaitPolicy, + Table: *n.desc.TableDesc(), + TableIsUncommittedVersion: n.desc.IsUncommittedVersion(), + Reverse: n.reverse, + IsCheck: n.isCheck, + Visibility: n.colCfg.visibility, + LockingStrength: n.lockingStrength, + LockingWaitPolicy: n.lockingWaitPolicy, // Retain the capacity of the spans slice. Spans: s.Spans[:0], @@ -1940,13 +1941,14 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( plan.AddProjection(pkCols) joinReaderSpec := execinfrapb.JoinReaderSpec{ - Table: *n.table.desc.TableDesc(), - IndexIdx: 0, - Visibility: n.table.colCfg.visibility, - LockingStrength: n.table.lockingStrength, - LockingWaitPolicy: n.table.lockingWaitPolicy, - MaintainOrdering: len(n.reqOrdering) > 0, - SystemColumns: n.table.systemColumns, + Table: *n.table.desc.TableDesc(), + TableIsUncommittedVersion: n.table.desc.IsUncommittedVersion(), + IndexIdx: 0, + Visibility: n.table.colCfg.visibility, + LockingStrength: n.table.lockingStrength, + LockingWaitPolicy: n.table.lockingWaitPolicy, + MaintainOrdering: len(n.reqOrdering) > 0, + SystemColumns: n.table.systemColumns, } post := execinfrapb.PostProcessSpec{ @@ -1998,13 +2000,14 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( } joinReaderSpec := execinfrapb.JoinReaderSpec{ - Table: *n.table.desc.TableDesc(), - Type: n.joinType, - Visibility: n.table.colCfg.visibility, - LockingStrength: n.table.lockingStrength, - LockingWaitPolicy: n.table.lockingWaitPolicy, - MaintainOrdering: len(n.reqOrdering) > 0, - SystemColumns: n.table.systemColumns, + Table: *n.table.desc.TableDesc(), + TableIsUncommittedVersion: n.table.desc.IsUncommittedVersion(), + Type: n.joinType, + Visibility: n.table.colCfg.visibility, + LockingStrength: n.table.lockingStrength, + LockingWaitPolicy: n.table.lockingWaitPolicy, + MaintainOrdering: len(n.reqOrdering) > 0, + SystemColumns: n.table.systemColumns, } joinReaderSpec.IndexIdx, err = getIndexIdx(n.table.index, n.table.desc) if err != nil { @@ -2180,11 +2183,14 @@ func (dsp *DistSQLPlanner) createPlanForZigzagJoin( plan = &p tables := make([]descpb.TableDescriptor, len(n.sides)) + tableIsUncommittedVersions := make([]bool, len(n.sides)) indexOrdinals := make([]uint32, len(n.sides)) cols := make([]execinfrapb.Columns, len(n.sides)) + numStreamCols := 0 for i, side := range n.sides { tables[i] = *side.scan.desc.TableDesc() + tableIsUncommittedVersions[i] = side.scan.desc.IsUncommittedVersion() indexOrdinals[i], err = getIndexIdx(side.scan.index, side.scan.desc) if err != nil { return nil, err @@ -2201,10 +2207,11 @@ func (dsp *DistSQLPlanner) createPlanForZigzagJoin( // The zigzag join node only represents inner joins, so hardcode Type to // InnerJoin. zigzagJoinerSpec := execinfrapb.ZigzagJoinerSpec{ - Tables: tables, - IndexOrdinals: indexOrdinals, - EqColumns: cols, - Type: descpb.InnerJoin, + Tables: tables, + TableIsUncommittedVersion: tableIsUncommittedVersions, + IndexOrdinals: indexOrdinals, + EqColumns: cols, + Type: descpb.InnerJoin, } zigzagJoinerSpec.FixedValues = make([]*execinfrapb.ValuesCoreSpec, len(n.sides)) diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 824fb29d57dc..884bf10828ca 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -232,10 +232,11 @@ func (e *distSQLSpecExecFactory) ConstructScan( colsToTableOrdinalMap := toTableOrdinals(cols, tabDesc, colCfg.visibility) trSpec := physicalplan.NewTableReaderSpec() *trSpec = execinfrapb.TableReaderSpec{ - Table: *tabDesc.TableDesc(), - Reverse: params.Reverse, - IsCheck: false, - Visibility: colCfg.visibility, + Table: *tabDesc.TableDesc(), + TableIsUncommittedVersion: tabDesc.IsUncommittedVersion(), + Reverse: params.Reverse, + IsCheck: false, + Visibility: colCfg.visibility, // Retain the capacity of the spans slice. Spans: trSpec.Spans[:0], SystemColumns: systemColumns, diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 6c99d471b209..e423efa53047 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_52afa52b11947d13, []int{0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -196,7 +196,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{12, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{12, 0} } type AggregatorSpec_Type int32 @@ -242,7 +242,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{12, 1} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{12, 1} } type WindowerSpec_WindowFunc int32 @@ -306,7 +306,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{15, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 0} } // Mode indicates which mode of framing is used. @@ -350,7 +350,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{15, 1, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1, 0} } // BoundType indicates which type of boundary is used. @@ -397,7 +397,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{15, 1, 1} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -440,7 +440,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_52afa52b11947d13, []int{15, 1, 2} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -460,7 +460,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_52afa52b11947d13, []int{0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -498,7 +498,8 @@ var xxx_messageInfo_ValuesCoreSpec proto.InternalMessageInfo // - Primary key as a string, if it was obtainable. // - JSON of all decoded column values. type TableReaderSpec struct { - Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + TableIsUncommittedVersion bool `protobuf:"varint,14,opt,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version"` // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. table.indexes[index_idx-1] IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -568,7 +569,7 @@ 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_52afa52b11947d13, []int{1} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -599,7 +600,8 @@ var xxx_messageInfo_TableReaderSpec proto.InternalMessageInfo // to skip unnecessary rows. This reader is used for different optimizations // when operating on a prefix of a compound key. type IndexSkipTableReaderSpec struct { - Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + TableIsUncommittedVersion bool `protobuf:"varint,8,opt,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version"` // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. table.indexes[index_idx-1] IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -623,7 +625,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_52afa52b11947d13, []int{2} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{2} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -671,7 +673,8 @@ var xxx_messageInfo_IndexSkipTableReaderSpec proto.InternalMessageInfo // If performing an index join (where a = c and b = d) (lookup columns is []): // Internal columns: | c | d | e | type JoinReaderSpec struct { - Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + TableIsUncommittedVersion bool `protobuf:"varint,13,opt,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version"` // If 0, we use the primary index; each row in the input stream has a value // for each primary key. The index must provide all lookup columns. IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -731,7 +734,7 @@ 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_52afa52b11947d13, []int{3} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{3} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -775,7 +778,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_52afa52b11947d13, []int{4} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{4} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -837,7 +840,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_52afa52b11947d13, []int{5} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{5} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -872,7 +875,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_52afa52b11947d13, []int{6} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{6} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -907,7 +910,8 @@ var xxx_messageInfo_OrdinalitySpec proto.InternalMessageInfo type ZigzagJoinerSpec struct { // TODO(pbardea): Replace these with inputs that conform to a RowSource-like // interface. - Tables []descpb.TableDescriptor `protobuf:"bytes,1,rep,name=tables" json:"tables"` + Tables []descpb.TableDescriptor `protobuf:"bytes,1,rep,name=tables" json:"tables"` + TableIsUncommittedVersion []bool `protobuf:"varint,7,rep,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version,omitempty"` // An array of arrays. The array at eq_columns[side_idx] contains the // equality columns for that side. All arrays in eq_columns should have // equal length. @@ -930,7 +934,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_52afa52b11947d13, []int{7} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{7} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1006,7 +1010,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_52afa52b11947d13, []int{8} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{8} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1093,7 +1097,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_52afa52b11947d13, []int{9} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{9} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1147,7 +1151,8 @@ var xxx_messageInfo_HashJoinerSpec proto.InternalMessageInfo // where d, e are not populated. // Internal columns for LEFT_SEMI and LEFT_ANTI: | a | b | type InvertedJoinerSpec struct { - Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table descpb.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + TableIsUncommittedVersion bool `protobuf:"varint,8,opt,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version"` // The ID of the inverted index. The first column in the index is the // inverted column, and the the remaining columns are the primary key. IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -1190,7 +1195,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_52afa52b11947d13, []int{10} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{10} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1243,7 +1248,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_52afa52b11947d13, []int{11} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{11} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1289,7 +1294,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_52afa52b11947d13, []int{12} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{12} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1340,7 +1345,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_52afa52b11947d13, []int{12, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{12, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1416,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_52afa52b11947d13, []int{13} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{13} } func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1437,7 +1442,8 @@ func (m *InterleavedReaderJoinerSpec) XXX_DiscardUnknown() { var xxx_messageInfo_InterleavedReaderJoinerSpec proto.InternalMessageInfo type InterleavedReaderJoinerSpec_Table struct { - Desc descpb.TableDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` + Desc descpb.TableDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` + TableIsUncommittedVersion bool `protobuf:"varint,6,opt,name=table_is_uncommitted_version,json=tableIsUncommittedVersion" json:"table_is_uncommitted_version"` // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. desc.indexes[index_idx-1] IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -1467,7 +1473,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_52afa52b11947d13, []int{13, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{13, 0} } func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1507,7 +1513,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_52afa52b11947d13, []int{14} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1549,7 +1555,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_52afa52b11947d13, []int{15} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1585,7 +1591,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_52afa52b11947d13, []int{15, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1621,7 +1627,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_52afa52b11947d13, []int{15, 1} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1662,7 +1668,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_52afa52b11947d13, []int{15, 1, 0} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1698,7 +1704,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_52afa52b11947d13, []int{15, 1, 1} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1748,7 +1754,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_52afa52b11947d13, []int{15, 2} + return fileDescriptor_processors_sql_94dd7e42a457c38c, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1930,6 +1936,14 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) } } + dAtA[i] = 0x70 + i++ + if m.TableIsUncommittedVersion { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -1988,6 +2002,14 @@ func (m *IndexSkipTableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x38 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LockingWaitPolicy)) + dAtA[i] = 0x40 + i++ + if m.TableIsUncommittedVersion { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -2077,6 +2099,14 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) } } + dAtA[i] = 0x68 + i++ + if m.TableIsUncommittedVersion { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -2250,6 +2280,18 @@ func (m *ZigzagJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x30 i++ i = encodeVarintProcessorsSql(dAtA, i, uint64(m.Type)) + if len(m.TableIsUncommittedVersion) > 0 { + for _, b := range m.TableIsUncommittedVersion { + dAtA[i] = 0x38 + i++ + if b { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + } return i, nil } @@ -2462,6 +2504,14 @@ func (m *InvertedJoinerSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x40 + i++ + if m.TableIsUncommittedVersion { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -2726,6 +2776,14 @@ func (m *InterleavedReaderJoinerSpec_Table) MarshalTo(dAtA []byte) (int, error) i += n } } + dAtA[i] = 0x30 + i++ + if m.TableIsUncommittedVersion { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ return i, nil } @@ -3065,6 +3123,7 @@ func (m *TableReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(e)) } } + n += 2 return n } @@ -3087,6 +3146,7 @@ func (m *IndexSkipTableReaderSpec) Size() (n int) { n += 2 n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) + n += 2 return n } @@ -3119,6 +3179,7 @@ func (m *JoinReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(e)) } } + n += 2 return n } @@ -3199,6 +3260,9 @@ func (m *ZigzagJoinerSpec) Size() (n int) { } } n += 1 + sovProcessorsSql(uint64(m.Type)) + if len(m.TableIsUncommittedVersion) > 0 { + n += 2 * len(m.TableIsUncommittedVersion) + } return n } @@ -3265,6 +3329,7 @@ func (m *InvertedJoinerSpec) Size() (n int) { n += 1 + l + sovProcessorsSql(uint64(l)) n += 1 + sovProcessorsSql(uint64(m.Type)) n += 2 + n += 2 return n } @@ -3376,6 +3441,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Size() (n int) { n += 1 + l + sovProcessorsSql(uint64(l)) } } + n += 2 return n } @@ -4002,6 +4068,26 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -4209,6 +4295,26 @@ func (m *IndexSkipTableReaderSpec) Unmarshal(dAtA []byte) error { break } } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -4593,6 +4699,26 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) } + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -5252,6 +5378,73 @@ func (m *ZigzagJoinerSpec) Unmarshal(dAtA []byte) error { break } } + case 7: + if wireType == 0 { + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = append(m.TableIsUncommittedVersion, bool(v != 0)) + } 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 + elementCount = packedLen + if elementCount != 0 && len(m.TableIsUncommittedVersion) == 0 { + m.TableIsUncommittedVersion = make([]bool, 0, elementCount) + } + for iNdEx < postIndex { + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = append(m.TableIsUncommittedVersion, bool(v != 0)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -5974,6 +6167,26 @@ func (m *InvertedJoinerSpec) Unmarshal(dAtA []byte) error { } } m.MaintainOrdering = bool(v != 0) + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -6929,6 +7142,26 @@ func (m *InterleavedReaderJoinerSpec_Table) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableIsUncommittedVersion", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TableIsUncommittedVersion = bool(v != 0) default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -8119,181 +8352,185 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_52afa52b11947d13) -} - -var fileDescriptor_processors_sql_52afa52b11947d13 = []byte{ - // 2740 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3a, 0xcd, 0x73, 0xdb, 0xc6, - 0xf5, 0x02, 0x3f, 0x24, 0xf2, 0xf1, 0x43, 0xeb, 0xb5, 0x12, 0x33, 0xcc, 0xef, 0x27, 0xcb, 0x4c, - 0x1a, 0xcb, 0x8e, 0x23, 0x35, 0x6a, 0x27, 0x9d, 0x24, 0x3d, 0x04, 0x24, 0x41, 0x99, 0x32, 0x05, - 0xd0, 0x20, 0x29, 0x3b, 0xc9, 0x4c, 0x30, 0x10, 0xb1, 0xa2, 0x10, 0x83, 0x00, 0x05, 0x80, 0x96, - 0x94, 0x7b, 0xa7, 0xd7, 0x76, 0x7a, 0xe9, 0xa9, 0x93, 0x4b, 0xfb, 0x17, 0xf4, 0xd6, 0x99, 0xf4, - 0xd2, 0x83, 0x8f, 0x39, 0xb5, 0x39, 0x75, 0x1a, 0xfb, 0x2f, 0x68, 0x2f, 0x99, 0xde, 0x3a, 0xbb, - 0x58, 0x40, 0x20, 0x47, 0x74, 0x4d, 0x5b, 0x89, 0xa7, 0x17, 0x0d, 0xf7, 0x7d, 0xed, 0xfb, 0xda, - 0xb7, 0xef, 0x2d, 0x04, 0x6f, 0x7a, 0x47, 0xd6, 0x26, 0x39, 0x21, 0x7d, 0xd3, 0x3e, 0x70, 0xf5, - 0xd1, 0xfe, 0xe6, 0xc8, 0x75, 0xfa, 0xc4, 0xf3, 0x1c, 0xd7, 0xd3, 0xbc, 0x23, 0x6b, 0x63, 0xe4, - 0x3a, 0xbe, 0x83, 0x4b, 0x7d, 0xa7, 0xff, 0xc0, 0x75, 0xf4, 0xfe, 0xe1, 0x06, 0x05, 0x1a, 0xa6, - 0xe7, 0x7b, 0x47, 0x96, 0x3b, 0xb6, 0xcb, 0x6f, 0x50, 0xfe, 0xbe, 0xee, 0xeb, 0x96, 0x33, 0xd8, - 0x34, 0x88, 0xd7, 0x1f, 0xed, 0x6f, 0x7a, 0xbe, 0x3b, 0xee, 0xfb, 0x63, 0x97, 0x18, 0x01, 0x7b, - 0xb9, 0x72, 0x0e, 0xd1, 0xe7, 0x8e, 0x69, 0x6b, 0xfe, 0xe9, 0x88, 0x70, 0x9a, 0xb5, 0x73, 0x68, - 0x2c, 0xa7, 0xff, 0xc0, 0xb4, 0x07, 0x9c, 0xe2, 0x15, 0x4a, 0x41, 0x39, 0xbc, 0xe0, 0x2f, 0x07, - 0x97, 0xa7, 0x2d, 0x30, 0x74, 0x5f, 0xe7, 0xb8, 0x1f, 0x3d, 0xc5, 0xba, 0x7d, 0xdd, 0x0b, 0xf7, - 0xbe, 0x49, 0xc9, 0x9c, 0x91, 0xbf, 0x69, 0xda, 0x0f, 0x89, 0xeb, 0x13, 0x83, 0x9c, 0x8c, 0xdc, - 0x4d, 0x6f, 0xa4, 0xdb, 0x1a, 0xfd, 0x45, 0x3c, 0xcf, 0x74, 0x6c, 0x4e, 0xbb, 0x32, 0x70, 0x06, - 0x0e, 0xfb, 0xb9, 0x49, 0x7f, 0x05, 0xd0, 0xca, 0x6f, 0x04, 0x28, 0xee, 0xe9, 0xd6, 0x98, 0x78, - 0x35, 0xc7, 0x25, 0x9d, 0x11, 0xe9, 0xe3, 0x1a, 0x2c, 0xf5, 0x1d, 0x6b, 0x3c, 0xb4, 0xbd, 0x92, - 0xb0, 0x96, 0x5c, 0xcf, 0x6d, 0xbd, 0xb1, 0x31, 0xcb, 0x8b, 0x1b, 0x75, 0xdd, 0x1f, 0x0f, 0x9b, - 0xf6, 0x81, 0x53, 0x4d, 0x3d, 0xfa, 0xfb, 0xd5, 0x05, 0x35, 0xe4, 0xc4, 0xaf, 0x43, 0xd6, 0xd5, - 0x8f, 0xb5, 0xfd, 0x53, 0x9f, 0x78, 0xa5, 0xc4, 0x5a, 0x72, 0x3d, 0xaf, 0x66, 0x5c, 0xfd, 0xb8, - 0x4a, 0xd7, 0xf8, 0x2a, 0x64, 0xec, 0xf1, 0x50, 0x73, 0x9d, 0x63, 0xaf, 0x94, 0x5c, 0x13, 0xd6, - 0x53, 0x21, 0xb7, 0x3d, 0x1e, 0xaa, 0xce, 0xb1, 0x57, 0xf9, 0x67, 0x1a, 0x96, 0xbb, 0xfa, 0xbe, - 0x45, 0x54, 0xa2, 0x1b, 0xc4, 0x65, 0x6a, 0x55, 0x21, 0xed, 0x53, 0x50, 0x49, 0x58, 0x13, 0xd6, - 0x73, 0x5b, 0x6f, 0x4d, 0x29, 0xe5, 0x1d, 0x59, 0xcc, 0x31, 0x8c, 0xad, 0x4e, 0xbc, 0xbe, 0x6b, - 0x8e, 0x7c, 0xc7, 0xe5, 0x92, 0x03, 0x56, 0x7c, 0x0d, 0xb2, 0xa6, 0x6d, 0x90, 0x13, 0xcd, 0x34, - 0x4e, 0x4a, 0x89, 0x35, 0x61, 0xbd, 0xc0, 0xf1, 0x19, 0x06, 0x6e, 0x1a, 0x27, 0x78, 0x15, 0x96, - 0x5c, 0xf2, 0x90, 0xb8, 0x1e, 0x61, 0xaa, 0x65, 0x42, 0xd5, 0x38, 0x10, 0x4b, 0x90, 0xa6, 0xfe, - 0xf5, 0x4a, 0x29, 0xe6, 0x9b, 0x1b, 0xb3, 0x7d, 0x33, 0x61, 0x80, 0x6e, 0x87, 0x9a, 0x30, 0x6e, - 0xfc, 0x06, 0x80, 0x65, 0x0e, 0x4d, 0x5f, 0x3b, 0x34, 0x6d, 0xbf, 0x94, 0x5e, 0x13, 0xd6, 0x93, - 0x9c, 0x20, 0xcb, 0xe0, 0xb7, 0x4d, 0xdb, 0xa7, 0x7e, 0x32, 0x3d, 0xad, 0x7f, 0x48, 0xfa, 0x0f, - 0x4a, 0x8b, 0x71, 0x65, 0x4c, 0xaf, 0x46, 0x81, 0x58, 0x06, 0x78, 0x68, 0x7a, 0xe6, 0xbe, 0x69, - 0x99, 0xfe, 0x69, 0x69, 0x69, 0x4d, 0x58, 0x2f, 0x6e, 0xad, 0xcf, 0xd6, 0xa8, 0xd3, 0xd7, 0xed, - 0xbd, 0x88, 0x9e, 0x0b, 0x8b, 0x49, 0xc0, 0x1f, 0xc2, 0x95, 0xa1, 0x7e, 0xa2, 0xf9, 0xe6, 0x90, - 0x78, 0xbe, 0x3e, 0x1c, 0x69, 0xfa, 0x80, 0x68, 0xb6, 0x6e, 0x3b, 0x5e, 0x29, 0x1b, 0x8b, 0xd3, - 0xca, 0x50, 0x3f, 0xe9, 0x86, 0x34, 0xe2, 0x80, 0xc8, 0x94, 0x02, 0x7f, 0x0a, 0x88, 0xe7, 0xbd, - 0xe6, 0xf9, 0x2e, 0xb1, 0x07, 0xfe, 0x61, 0x09, 0x98, 0x4a, 0x37, 0x67, 0xc4, 0x8a, 0xea, 0xd3, - 0x0a, 0x58, 0x3a, 0x9c, 0x83, 0xef, 0xb0, 0x6c, 0x4d, 0x82, 0xf1, 0x3e, 0x5c, 0x0e, 0x85, 0x1f, - 0xeb, 0xa6, 0xaf, 0x8d, 0x1c, 0xcb, 0xec, 0x9f, 0x96, 0x72, 0x4c, 0xfe, 0xad, 0xff, 0x2e, 0xff, - 0x9e, 0x6e, 0xfa, 0x6d, 0xc6, 0xc3, 0x77, 0xb8, 0x64, 0x4d, 0x23, 0xf0, 0x5b, 0x90, 0x1b, 0xe9, - 0xae, 0x6e, 0x59, 0xc4, 0x32, 0xbf, 0x20, 0xa5, 0x7c, 0xcc, 0xe3, 0x71, 0x04, 0x96, 0xa1, 0xe8, - 0x9d, 0x7a, 0x3e, 0x19, 0x6a, 0xe1, 0x39, 0x29, 0xac, 0x25, 0xd7, 0x8b, 0x5b, 0xd7, 0x67, 0xa9, - 0xc1, 0x88, 0x6b, 0x8c, 0xf6, 0x8e, 0x69, 0x1b, 0x6a, 0xc1, 0x8b, 0x41, 0xbc, 0x9d, 0x54, 0x26, - 0x83, 0xb2, 0x95, 0x7f, 0x27, 0xa1, 0xd4, 0xa4, 0x59, 0xd8, 0x79, 0x60, 0x8e, 0x5e, 0x52, 0xf2, - 0x47, 0xc9, 0x9d, 0x7c, 0xa1, 0xe4, 0x9e, 0x4c, 0xcb, 0xd4, 0x0b, 0xa7, 0x65, 0xec, 0x4c, 0xa6, - 0xcf, 0x3b, 0x93, 0xe7, 0x65, 0xde, 0xe2, 0xf7, 0x9c, 0x79, 0x4b, 0x17, 0x98, 0x79, 0x95, 0xef, - 0xd2, 0x50, 0xdc, 0x71, 0x4c, 0xfb, 0x87, 0x8f, 0xf8, 0x0d, 0x28, 0x5a, 0x8e, 0xf3, 0x60, 0x3c, - 0x8a, 0x72, 0x99, 0x86, 0xbe, 0x50, 0x4d, 0x20, 0x41, 0x2d, 0x04, 0x18, 0x9e, 0xa6, 0xf4, 0x5e, - 0x70, 0x82, 0x7b, 0x85, 0x85, 0x34, 0xb7, 0xf5, 0xe6, 0xec, 0x90, 0x4a, 0xd1, 0xed, 0xc3, 0x77, - 0x5c, 0x74, 0x6c, 0x0a, 0xc3, 0xef, 0x43, 0x8a, 0xde, 0x81, 0x3c, 0x3c, 0x57, 0x67, 0x58, 0x45, - 0x7d, 0xd1, 0x3d, 0x1d, 0x11, 0xce, 0xcc, 0x58, 0x2e, 0xbc, 0xd8, 0xbd, 0x0f, 0xaf, 0x4e, 0x9a, - 0xae, 0xe9, 0x2e, 0xd1, 0x1e, 0x90, 0xd3, 0x52, 0x26, 0x96, 0x64, 0x97, 0x27, 0x9c, 0x20, 0xba, - 0xe4, 0x0e, 0x39, 0x3d, 0x37, 0xe1, 0xb2, 0xdf, 0x73, 0xc2, 0xc1, 0x45, 0x96, 0xba, 0x77, 0xe1, - 0xd2, 0x50, 0x37, 0x6d, 0x5f, 0x37, 0x6d, 0xcd, 0x71, 0x0d, 0xe2, 0x9a, 0xf6, 0x80, 0x15, 0xd3, - 0xd0, 0x6c, 0x14, 0xa2, 0x15, 0x8e, 0x3d, 0xa7, 0xea, 0xe5, 0x5f, 0xb0, 0xea, 0xa5, 0xd1, 0x22, - 0xed, 0x3f, 0xa0, 0xe3, 0xb8, 0x3e, 0xcf, 0xfa, 0xbb, 0xb0, 0xec, 0x8c, 0xfd, 0xd1, 0xd8, 0x3f, - 0xd3, 0x2a, 0xc8, 0xff, 0xca, 0xec, 0x40, 0x87, 0x1a, 0x72, 0xcd, 0x8b, 0x81, 0x80, 0x48, 0xef, - 0x2d, 0xc0, 0xa1, 0x2c, 0x6d, 0xa8, 0xfb, 0xfd, 0x43, 0xcd, 0x22, 0xf6, 0xc4, 0x69, 0x40, 0x21, - 0x7e, 0x97, 0xa2, 0x5b, 0xc4, 0xae, 0x7c, 0x25, 0x40, 0xbe, 0x6e, 0x7a, 0xbe, 0x69, 0xf7, 0x7d, - 0xa6, 0xd7, 0x75, 0x58, 0x66, 0x44, 0xc4, 0xd0, 0xe2, 0xbd, 0x51, 0x41, 0x2d, 0x72, 0x70, 0x78, - 0x48, 0x6e, 0x00, 0x32, 0x38, 0x63, 0x44, 0x99, 0x60, 0x94, 0xcb, 0x21, 0x3c, 0x24, 0xdd, 0x02, - 0x6c, 0x8f, 0x2d, 0x2b, 0x48, 0xbb, 0x10, 0x39, 0xd1, 0x74, 0x20, 0x86, 0x17, 0x5d, 0x12, 0xea, - 0x82, 0xdf, 0x82, 0x3c, 0x71, 0x5d, 0xc7, 0xd5, 0x1c, 0x5b, 0x33, 0xc6, 0x23, 0x76, 0x10, 0xb3, - 0x61, 0x6e, 0x33, 0x8c, 0x62, 0xd7, 0xc7, 0xa3, 0x0a, 0x82, 0xa2, 0xe2, 0x1a, 0xa6, 0xad, 0xd3, - 0x4c, 0xa7, 0x16, 0x54, 0x7e, 0x9b, 0x04, 0xf4, 0x89, 0x39, 0xf8, 0x42, 0x1f, 0xd0, 0xc3, 0xc5, - 0xdd, 0x5d, 0x87, 0x45, 0x56, 0x29, 0xc2, 0x4e, 0x6f, 0xbe, 0x2a, 0xc3, 0x79, 0x71, 0x03, 0x80, - 0x1c, 0x4d, 0x58, 0x9b, 0xdb, 0xba, 0x36, 0x3b, 0x5e, 0xdc, 0xfe, 0xb0, 0xdd, 0x21, 0x47, 0x67, - 0xbe, 0x2b, 0x06, 0xe5, 0xca, 0x09, 0x54, 0x9f, 0xa8, 0x45, 0x0c, 0xc3, 0x6d, 0xba, 0xa0, 0x5a, - 0x74, 0x07, 0xf2, 0x07, 0xe6, 0x09, 0x31, 0xb4, 0x87, 0xac, 0x01, 0x2e, 0xa5, 0x99, 0xe6, 0x4f, - 0x29, 0x29, 0x93, 0x8d, 0xb2, 0x9a, 0x63, 0xdc, 0x01, 0xf0, 0x05, 0x0a, 0x5b, 0xe5, 0xaf, 0x49, - 0x58, 0xde, 0x25, 0xee, 0x80, 0xc4, 0x22, 0xb3, 0x0b, 0x05, 0x8b, 0x1c, 0xbc, 0xc0, 0x31, 0xc8, - 0x53, 0xf6, 0xe8, 0x10, 0x28, 0x50, 0x74, 0xcd, 0xc1, 0x61, 0x4c, 0x5e, 0x62, 0x4e, 0x79, 0x05, - 0xc6, 0x1f, 0x09, 0x8c, 0x05, 0x20, 0xfd, 0x32, 0x2e, 0x83, 0x1b, 0x50, 0xa0, 0x87, 0x43, 0x23, - 0x47, 0x63, 0x3d, 0xba, 0x0f, 0xc2, 0x73, 0x93, 0xa7, 0x28, 0x89, 0x63, 0xf0, 0x07, 0x70, 0x85, - 0xb9, 0xf2, 0x2c, 0x47, 0x67, 0x14, 0x7a, 0x72, 0xe0, 0x4b, 0x47, 0x93, 0x85, 0xfe, 0xe7, 0x50, - 0x0a, 0xfc, 0x76, 0x0e, 0x73, 0x36, 0xc6, 0xbc, 0xc2, 0xa8, 0xa6, 0xb8, 0x2b, 0xbf, 0x4e, 0x42, - 0xf1, 0xb6, 0xee, 0x1d, 0xc6, 0xe2, 0x7a, 0x13, 0x96, 0xa7, 0x94, 0x09, 0x0a, 0x09, 0xbf, 0x70, - 0xe3, 0x2a, 0xe0, 0x5b, 0x80, 0xa6, 0x37, 0x0f, 0x6a, 0x09, 0x23, 0x2e, 0x4e, 0x6e, 0xf9, 0xd2, - 0x23, 0xf2, 0x36, 0x14, 0x87, 0x34, 0x89, 0xcf, 0x2a, 0x64, 0x3c, 0x24, 0x85, 0x00, 0x17, 0x2a, - 0xfb, 0xf2, 0x62, 0xf2, 0x8b, 0x24, 0xe0, 0x26, 0x9f, 0x96, 0x63, 0x71, 0xf9, 0x81, 0xda, 0x2d, - 0x05, 0x0a, 0xe1, 0xa8, 0xfe, 0xbc, 0xd5, 0x2b, 0x1f, 0x0a, 0x60, 0x01, 0x7b, 0xd9, 0x51, 0x3f, - 0xb7, 0x91, 0x58, 0x7a, 0x5a, 0x23, 0xb1, 0x93, 0xca, 0x24, 0x51, 0xaa, 0xf2, 0x07, 0x01, 0x56, - 0xc2, 0x38, 0x34, 0x4c, 0xcb, 0x27, 0x2e, 0x8f, 0xc4, 0x06, 0xa0, 0xc8, 0x45, 0x7d, 0xc7, 0x62, - 0xce, 0x14, 0x62, 0xce, 0x2c, 0x86, 0xd8, 0x9a, 0x63, 0x51, 0x97, 0x7e, 0x36, 0xed, 0xd2, 0xa0, - 0xb2, 0xfd, 0x64, 0xca, 0x0a, 0x67, 0xe4, 0x6f, 0xc4, 0x5f, 0x49, 0x36, 0xe8, 0xd8, 0x72, 0xe6, - 0x91, 0xb6, 0xeb, 0xf8, 0xce, 0x79, 0x1e, 0xae, 0xfc, 0x29, 0x03, 0x45, 0x71, 0x30, 0x70, 0xc9, - 0x40, 0xf7, 0x9d, 0x40, 0xc5, 0x6b, 0x00, 0x03, 0xd7, 0x09, 0x1a, 0xc7, 0xf8, 0x91, 0xcc, 0x32, - 0x68, 0xcd, 0xb1, 0x3c, 0xfc, 0x19, 0xe4, 0x75, 0xce, 0x64, 0x3a, 0xd1, 0x40, 0xf5, 0xd3, 0xd9, - 0xc1, 0x99, 0xdc, 0x22, 0x5a, 0xc6, 0xe2, 0x1e, 0x97, 0x87, 0x7f, 0xcc, 0xbb, 0x1a, 0x62, 0x68, - 0x31, 0x55, 0x52, 0x91, 0x2a, 0x88, 0x63, 0xb7, 0x23, 0x8d, 0xb6, 0x79, 0x90, 0xd3, 0x2c, 0xc8, - 0xef, 0x3c, 0xb3, 0x26, 0xd3, 0x21, 0x2f, 0xff, 0x32, 0x01, 0xb9, 0x98, 0x7a, 0x54, 0xf0, 0xc1, - 0xd8, 0xee, 0xb3, 0x20, 0xcd, 0x23, 0xb8, 0x31, 0xb6, 0xfb, 0xa1, 0x60, 0x2a, 0x00, 0xaf, 0x41, - 0x26, 0x6a, 0x83, 0x12, 0xb1, 0x14, 0x8a, 0xa0, 0xf8, 0x4d, 0x28, 0x1e, 0xb0, 0x5c, 0x89, 0x32, - 0x83, 0x9e, 0x9f, 0x82, 0x9a, 0x0f, 0xa0, 0x3c, 0x23, 0xae, 0xb0, 0x07, 0x2c, 0x86, 0x4e, 0xb3, - 0xd6, 0x6b, 0xb1, 0x1f, 0x20, 0x6e, 0x43, 0x56, 0x77, 0x07, 0xe3, 0x21, 0xb1, 0x7d, 0xaf, 0xb4, - 0xc8, 0x22, 0x32, 0xcf, 0x71, 0x39, 0x63, 0xe6, 0x39, 0xfc, 0xaf, 0x24, 0xa4, 0xa8, 0x15, 0x18, - 0x41, 0x5e, 0x94, 0x3f, 0xd6, 0x64, 0xa5, 0xab, 0xc9, 0xbd, 0x56, 0x0b, 0x2d, 0xe0, 0x25, 0x48, - 0x8a, 0x7b, 0xdb, 0x48, 0xc0, 0x79, 0xc8, 0x54, 0x15, 0xa5, 0xa5, 0x89, 0x72, 0x1d, 0x25, 0x70, - 0x0e, 0x96, 0xd8, 0x4a, 0x51, 0x51, 0x12, 0x17, 0x01, 0x6a, 0x8a, 0x5c, 0x13, 0xbb, 0x9a, 0xb8, - 0xbd, 0x8d, 0x52, 0x38, 0x0b, 0xe9, 0x9a, 0xd2, 0x93, 0xbb, 0x28, 0x4d, 0xd9, 0x77, 0xc5, 0xfb, - 0x68, 0x89, 0xfd, 0x68, 0xca, 0x28, 0x83, 0x01, 0x16, 0x3b, 0xdd, 0x7a, 0x5d, 0xda, 0x43, 0x59, - 0x0a, 0xec, 0xf4, 0x76, 0x11, 0x50, 0x71, 0x9d, 0xde, 0xae, 0xd6, 0x94, 0xbb, 0x28, 0x47, 0x77, - 0xda, 0x13, 0xd5, 0xa6, 0x28, 0xd7, 0x24, 0x94, 0xa7, 0xa8, 0xfb, 0x8a, 0xca, 0x24, 0x17, 0x82, - 0x9d, 0x7a, 0x72, 0x57, 0x53, 0x95, 0x7b, 0x1d, 0x54, 0x64, 0x7c, 0x77, 0xd5, 0x7a, 0xb3, 0xd1, - 0x40, 0xcb, 0x18, 0x43, 0xb1, 0xd1, 0x94, 0xc5, 0x96, 0x16, 0x71, 0x23, 0x6a, 0x50, 0x00, 0xe3, - 0x7b, 0x5e, 0xc2, 0x05, 0xc8, 0x8a, 0xaa, 0x2a, 0x7e, 0xcc, 0x24, 0x62, 0xba, 0xd9, 0x4e, 0x47, - 0x91, 0xd9, 0xea, 0x32, 0x45, 0xd2, 0x55, 0x95, 0x2d, 0x57, 0xe8, 0x76, 0x9d, 0xae, 0xda, 0x94, - 0xb7, 0xd9, 0xfa, 0x15, 0x66, 0x75, 0xb3, 0xcb, 0x5c, 0xf0, 0x2a, 0x35, 0x84, 0x2e, 0x14, 0x15, - 0x5d, 0xc1, 0x19, 0x48, 0xd5, 0x14, 0x55, 0x45, 0x25, 0x5c, 0x82, 0x95, 0xb6, 0xa4, 0xd6, 0x24, - 0xb9, 0xdb, 0x6c, 0x49, 0x5a, 0xbd, 0xd9, 0xa9, 0x69, 0xcd, 0xdd, 0x76, 0x0b, 0xbd, 0x36, 0x85, - 0xa9, 0x29, 0x72, 0x37, 0xc0, 0x94, 0xf1, 0x65, 0x58, 0x66, 0x3a, 0x28, 0xd5, 0x1d, 0xa9, 0x16, - 0x38, 0xf1, 0x75, 0xbc, 0x02, 0x28, 0x50, 0x25, 0x06, 0xfd, 0x3f, 0xaa, 0xc1, 0x9e, 0xa8, 0x6a, - 0x6d, 0xa5, 0x8d, 0xfe, 0x3f, 0x50, 0x8f, 0x9a, 0xc5, 0xd6, 0xab, 0x78, 0x19, 0x72, 0x9d, 0xae, - 0xb6, 0x2b, 0xde, 0x91, 0x5a, 0x4d, 0x59, 0x42, 0x57, 0xa9, 0x39, 0x9d, 0xae, 0x26, 0xdd, 0xef, - 0x4a, 0x72, 0x17, 0xad, 0x51, 0x5b, 0x3b, 0x5d, 0xad, 0x27, 0x37, 0x15, 0x19, 0x5d, 0xab, 0xdc, - 0x82, 0x14, 0x3d, 0x12, 0x54, 0x77, 0xb1, 0xd7, 0x55, 0xd0, 0x02, 0x0b, 0x4d, 0x4d, 0x6c, 0x89, - 0x2a, 0x12, 0xa8, 0x6c, 0x59, 0x91, 0x35, 0xbe, 0x4e, 0x54, 0xfe, 0xbc, 0x08, 0xaf, 0x37, 0x6d, - 0x9f, 0xb8, 0x16, 0xd1, 0x1f, 0x12, 0x23, 0x18, 0xf0, 0x63, 0xf7, 0xce, 0xc7, 0x53, 0x1d, 0xf8, - 0x87, 0xb3, 0xf3, 0xf1, 0x29, 0x62, 0x82, 0x7b, 0x69, 0xaa, 0x2d, 0x8f, 0xbd, 0x9a, 0x24, 0xce, - 0x7b, 0x35, 0x99, 0x7c, 0x82, 0x4c, 0x9e, 0xff, 0x04, 0x79, 0xa1, 0x43, 0x7f, 0x7a, 0xfe, 0xfb, - 0xe5, 0x7f, 0xfd, 0x69, 0xa7, 0xfc, 0x55, 0x02, 0xd2, 0x2c, 0x3a, 0xf8, 0x23, 0x48, 0x19, 0xc4, - 0xeb, 0x3f, 0x57, 0x87, 0xc1, 0x38, 0x9f, 0xa5, 0xc1, 0xa8, 0x41, 0x6a, 0xe4, 0x78, 0x41, 0x38, - 0x9f, 0xfa, 0x80, 0xd7, 0x76, 0x3c, 0xbf, 0x1d, 0x7c, 0x54, 0xa0, 0x19, 0x14, 0xee, 0x43, 0x99, - 0x71, 0x1d, 0x32, 0xd1, 0x5d, 0x9e, 0x9a, 0x73, 0x4e, 0x88, 0x38, 0xcf, 0x1e, 0x13, 0xd3, 0x2f, - 0xf2, 0x98, 0x58, 0xf9, 0x8b, 0x00, 0xc5, 0xb6, 0xeb, 0x7c, 0x4e, 0xfa, 0x7e, 0x87, 0x04, 0xd3, - 0xf8, 0x47, 0x90, 0xa6, 0x19, 0x19, 0x9e, 0x99, 0x79, 0x52, 0x32, 0x60, 0xc4, 0xdb, 0x70, 0x69, - 0x40, 0x6c, 0xe2, 0xea, 0x7e, 0xac, 0x5f, 0x0d, 0x26, 0xd7, 0xf2, 0x74, 0x60, 0xc8, 0x70, 0x23, - 0xf8, 0x6c, 0xd3, 0x55, 0x51, 0xc4, 0x14, 0x36, 0xb2, 0xd7, 0x01, 0xd9, 0x63, 0xf6, 0x24, 0xe2, - 0x69, 0x23, 0xe2, 0x6a, 0x03, 0x62, 0x07, 0x53, 0xab, 0x5a, 0xb0, 0xc7, 0x43, 0x7a, 0xf1, 0xb6, - 0x89, 0xbb, 0x4d, 0xec, 0xca, 0xb7, 0x05, 0xc8, 0xdf, 0x33, 0x6d, 0xc3, 0x39, 0xe6, 0x27, 0x7f, - 0x8d, 0xbd, 0x36, 0xfb, 0x26, 0xbb, 0xe2, 0x4f, 0xf9, 0x73, 0x42, 0x1c, 0x84, 0x3b, 0x90, 0x3d, - 0x66, 0x1c, 0x8d, 0x48, 0xb9, 0xcd, 0xd9, 0xa6, 0xc6, 0x85, 0xf3, 0x45, 0x23, 0xba, 0xb9, 0x22, - 0x39, 0xe5, 0x3f, 0x0a, 0xfc, 0xce, 0xea, 0x40, 0x21, 0xec, 0x28, 0x48, 0xe3, 0x79, 0xef, 0x6f, - 0x75, 0x52, 0x06, 0xbe, 0x0b, 0xc0, 0xb7, 0xa2, 0x12, 0x13, 0x4c, 0xe2, 0xbb, 0xf3, 0xe9, 0x4c, - 0xa5, 0xc6, 0x84, 0x7c, 0x90, 0x7a, 0xf4, 0xe5, 0x55, 0xa1, 0xfc, 0xe5, 0x12, 0xa4, 0x1b, 0xae, - 0x3e, 0x24, 0xf8, 0x0e, 0xa4, 0x86, 0x8e, 0x41, 0xb8, 0xba, 0xcf, 0x2a, 0x9c, 0xf1, 0x6e, 0xec, - 0x3a, 0x46, 0x54, 0x5e, 0xa8, 0x10, 0x7c, 0x17, 0x16, 0xf7, 0x9d, 0xb1, 0x6d, 0x78, 0x33, 0xba, - 0xc6, 0xa7, 0x8b, 0xab, 0x32, 0xd6, 0xb0, 0xd8, 0x05, 0x82, 0xf0, 0x27, 0x90, 0x25, 0x27, 0x7d, - 0x6b, 0x4c, 0x93, 0x8e, 0x1d, 0xc3, 0xe2, 0xd6, 0x7b, 0x73, 0x49, 0x95, 0x42, 0xee, 0xe8, 0x85, - 0x24, 0x04, 0x94, 0xbf, 0x13, 0x20, 0xcd, 0x36, 0xa5, 0xbb, 0xb0, 0xfd, 0x68, 0xc1, 0xe4, 0xae, - 0x78, 0x6f, 0x7e, 0xdd, 0x63, 0xe5, 0xf6, 0x4c, 0x1c, 0xbd, 0x18, 0x4c, 0xdb, 0xd7, 0x9c, 0x83, - 0x03, 0x8f, 0x04, 0x9d, 0x58, 0xf8, 0xe1, 0x27, 0x6b, 0xda, 0xbe, 0xc2, 0xc0, 0xf8, 0x1a, 0xe4, - 0xe9, 0xa9, 0x30, 0x42, 0x32, 0x6a, 0x69, 0x5e, 0xcd, 0x31, 0x18, 0x27, 0xd9, 0x81, 0x5c, 0x80, - 0x64, 0x9f, 0x4b, 0x79, 0x25, 0x99, 0xe3, 0x63, 0x22, 0x04, 0xdc, 0x54, 0xa7, 0xf2, 0xef, 0x04, - 0x58, 0x0c, 0xdc, 0x8d, 0x65, 0x48, 0x7b, 0xbe, 0xee, 0xfa, 0xbc, 0x90, 0x6e, 0xcd, 0x6f, 0x76, - 0x54, 0x60, 0xa8, 0x18, 0x5c, 0x87, 0x24, 0xb1, 0x0d, 0x9e, 0x00, 0xcf, 0x21, 0x4d, 0xa5, 0xec, - 0x95, 0xeb, 0x90, 0xa2, 0xd9, 0x45, 0x9b, 0x38, 0x55, 0x94, 0xb7, 0x25, 0xb4, 0x40, 0x3b, 0x04, - 0xd6, 0x6f, 0x09, 0xb4, 0x43, 0xd8, 0x56, 0x95, 0x5e, 0xbb, 0x83, 0x12, 0x95, 0x2f, 0x20, 0x1b, - 0xf9, 0x1e, 0x5f, 0x81, 0xcb, 0x3d, 0xb9, 0xaa, 0xf4, 0xe4, 0xba, 0x54, 0xd7, 0xda, 0xaa, 0x54, - 0x93, 0xea, 0x4d, 0x79, 0x1b, 0x2d, 0x4c, 0x22, 0x1a, 0x4a, 0xab, 0xa5, 0xdc, 0xa3, 0x08, 0x81, - 0xf6, 0x37, 0x4a, 0xa3, 0xd1, 0x91, 0xba, 0x31, 0xf2, 0x44, 0x0c, 0x7a, 0x46, 0x9b, 0xa4, 0x8d, - 0x4d, 0xad, 0xa7, 0xaa, 0x52, 0xd0, 0xf8, 0xa1, 0x54, 0xe5, 0x53, 0xc8, 0x46, 0xd9, 0x45, 0x7b, - 0x3c, 0x59, 0xd1, 0xa4, 0xfb, 0xb5, 0x56, 0xaf, 0x43, 0x5b, 0x1b, 0xb6, 0x29, 0x5b, 0xd6, 0x25, - 0x2d, 0xce, 0x27, 0xe0, 0x4b, 0x50, 0x08, 0x11, 0xcc, 0x0e, 0x94, 0xa0, 0xdc, 0x21, 0xa8, 0xdb, - 0x94, 0x3a, 0x28, 0x59, 0xfe, 0x5b, 0x02, 0x32, 0x61, 0xdd, 0xc1, 0x52, 0x6c, 0x28, 0xc8, 0x6d, - 0xbd, 0xfd, 0xac, 0x5e, 0x9d, 0x1e, 0x09, 0x2e, 0xe6, 0x26, 0xaa, 0x42, 0xfa, 0x80, 0xc6, 0x8b, - 0x8f, 0xc8, 0xb7, 0xe6, 0x89, 0xb1, 0x1a, 0xb0, 0xe2, 0x75, 0x98, 0x18, 0x32, 0x58, 0x13, 0x92, - 0x0e, 0x47, 0xb3, 0x89, 0xf1, 0xa3, 0x0c, 0x19, 0xdd, 0x1d, 0x78, 0x4d, 0xe3, 0xc4, 0x2b, 0x2d, - 0xb1, 0xaa, 0x1e, 0xad, 0xa9, 0x94, 0xe0, 0x79, 0x9a, 0x4b, 0xc9, 0xc4, 0x2e, 0xf1, 0x09, 0xcc, - 0x4e, 0x2a, 0x93, 0x40, 0x49, 0x3e, 0x67, 0xfc, 0x5e, 0x00, 0x38, 0xab, 0x8e, 0xb4, 0xc7, 0x54, - 0x95, 0x7b, 0x9a, 0xdc, 0xdb, 0xad, 0x4a, 0x2a, 0xcf, 0x33, 0x51, 0xbe, 0x13, 0x74, 0x9f, 0x75, - 0x49, 0xee, 0x48, 0x1a, 0x5b, 0xb3, 0x20, 0xf1, 0xde, 0x39, 0x80, 0x24, 0x69, 0x6b, 0x5b, 0xeb, - 0xed, 0xb2, 0x0e, 0xbb, 0x1b, 0x8c, 0x1c, 0xac, 0xaf, 0x0e, 0x46, 0x8e, 0x96, 0xb8, 0x8d, 0x16, - 0xa9, 0xb8, 0x96, 0x24, 0xd6, 0xd1, 0x12, 0xcd, 0x9f, 0x46, 0x53, 0xed, 0x74, 0xb5, 0x3d, 0xb1, - 0xd5, 0x93, 0x50, 0x86, 0xca, 0x6f, 0x89, 0xd1, 0x3a, 0x4b, 0xa5, 0xc9, 0xdd, 0xdb, 0x7c, 0x09, - 0x37, 0x7f, 0x06, 0xc5, 0xc9, 0xaf, 0x2e, 0x34, 0xf1, 0xdb, 0xbd, 0x6a, 0xab, 0x59, 0x43, 0x0b, - 0xf8, 0x35, 0x78, 0x25, 0xf8, 0x4d, 0x07, 0x01, 0x36, 0x2b, 0x71, 0x94, 0x50, 0x7d, 0xe7, 0xd1, - 0xb7, 0xab, 0x0b, 0x8f, 0x1e, 0xaf, 0x0a, 0x5f, 0x3f, 0x5e, 0x15, 0xbe, 0x79, 0xbc, 0x2a, 0xfc, - 0xe3, 0xf1, 0xaa, 0xf0, 0xab, 0x27, 0xab, 0x0b, 0x5f, 0x3f, 0x59, 0x5d, 0xf8, 0xe6, 0xc9, 0xea, - 0xc2, 0x27, 0xb9, 0xd8, 0x3f, 0x43, 0xfc, 0x27, 0x00, 0x00, 0xff, 0xff, 0x2a, 0xd5, 0x28, 0xb1, - 0xfc, 0x21, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_94dd7e42a457c38c) +} + +var fileDescriptor_processors_sql_94dd7e42a457c38c = []byte{ + // 2806 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x1a, 0x4b, 0x73, 0xdb, 0xc6, + 0x59, 0xe0, 0x4b, 0xe4, 0xc7, 0x87, 0xd6, 0x6b, 0x27, 0x66, 0x94, 0x54, 0x96, 0x99, 0x34, 0x96, + 0x1d, 0x47, 0x6a, 0xd4, 0x4e, 0x3a, 0x49, 0x3a, 0xd3, 0x40, 0x24, 0x28, 0x53, 0xa6, 0x00, 0x19, + 0x24, 0x65, 0x27, 0x99, 0x09, 0x06, 0x02, 0x56, 0x14, 0x62, 0x10, 0xa0, 0x00, 0xd0, 0x92, 0x72, + 0x6f, 0x7b, 0x6d, 0xa7, 0xf7, 0x4e, 0x2e, 0xed, 0x2f, 0xe8, 0xa1, 0x33, 0x9d, 0xc9, 0xf4, 0xd0, + 0x83, 0x0f, 0x3d, 0xe4, 0xd4, 0xa6, 0x97, 0x4e, 0xe3, 0xfc, 0x83, 0x5e, 0x7a, 0xed, 0xec, 0x62, + 0x01, 0x81, 0x1c, 0x51, 0x96, 0x6c, 0x25, 0x9e, 0x5e, 0x34, 0xdc, 0xef, 0xb5, 0xdf, 0x7b, 0xbf, + 0x5d, 0x08, 0xde, 0xf0, 0xf7, 0xed, 0x15, 0x72, 0x48, 0x0c, 0xcb, 0xd9, 0xf5, 0xf4, 0xe1, 0xce, + 0xca, 0xd0, 0x73, 0x0d, 0xe2, 0xfb, 0xae, 0xe7, 0x6b, 0xfe, 0xbe, 0xbd, 0x3c, 0xf4, 0xdc, 0xc0, + 0xc5, 0x55, 0xc3, 0x35, 0x1e, 0x7a, 0xae, 0x6e, 0xec, 0x2d, 0x53, 0xa0, 0x69, 0xf9, 0x81, 0xbf, + 0x6f, 0x7b, 0x23, 0x67, 0xfe, 0x75, 0xca, 0x6f, 0xe8, 0x81, 0x6e, 0xbb, 0xfd, 0x15, 0x93, 0xf8, + 0xc6, 0x70, 0x67, 0xc5, 0x0f, 0xbc, 0x91, 0x11, 0x8c, 0x3c, 0x62, 0x86, 0xec, 0xf3, 0xb5, 0x13, + 0x88, 0x3e, 0x73, 0x2d, 0x47, 0x0b, 0x8e, 0x86, 0x84, 0xd3, 0x2c, 0x9e, 0x40, 0x63, 0xbb, 0xc6, + 0x43, 0xcb, 0xe9, 0x73, 0x8a, 0x97, 0x28, 0x05, 0xe5, 0xf0, 0xc3, 0xbf, 0x1c, 0x3c, 0x3f, 0x69, + 0x81, 0xa9, 0x07, 0x3a, 0xc7, 0xfd, 0xf0, 0x14, 0xeb, 0x76, 0x74, 0x3f, 0xda, 0xfb, 0x16, 0x25, + 0x73, 0x87, 0xc1, 0x8a, 0xe5, 0x3c, 0x22, 0x5e, 0x40, 0x4c, 0x72, 0x38, 0xf4, 0x56, 0xfc, 0xa1, + 0xee, 0x68, 0xf4, 0x17, 0xf1, 0x7d, 0xcb, 0x75, 0x38, 0xed, 0x95, 0xbe, 0xdb, 0x77, 0xd9, 0xcf, + 0x15, 0xfa, 0x2b, 0x84, 0xd6, 0x7e, 0x2b, 0x40, 0x65, 0x5b, 0xb7, 0x47, 0xc4, 0xaf, 0xbb, 0x1e, + 0xe9, 0x0c, 0x89, 0x81, 0xeb, 0x30, 0x6b, 0xb8, 0xf6, 0x68, 0xe0, 0xf8, 0x55, 0x61, 0x31, 0xbd, + 0x54, 0x5c, 0x7d, 0x7d, 0x79, 0x9a, 0x17, 0x97, 0x1b, 0x7a, 0x30, 0x1a, 0xb4, 0x9c, 0x5d, 0x77, + 0x2d, 0xf3, 0xf8, 0x5f, 0xd7, 0x66, 0xd4, 0x88, 0x13, 0xbf, 0x0a, 0x05, 0x4f, 0x3f, 0xd0, 0x76, + 0x8e, 0x02, 0xe2, 0x57, 0x53, 0x8b, 0xe9, 0xa5, 0x92, 0x9a, 0xf7, 0xf4, 0x83, 0x35, 0xba, 0xc6, + 0xd7, 0x20, 0xef, 0x8c, 0x06, 0x9a, 0xe7, 0x1e, 0xf8, 0xd5, 0xf4, 0xa2, 0xb0, 0x94, 0x89, 0xb8, + 0x9d, 0xd1, 0x40, 0x75, 0x0f, 0xfc, 0xda, 0x9f, 0x72, 0x30, 0xd7, 0xd5, 0x77, 0x6c, 0xa2, 0x12, + 0xdd, 0x24, 0x1e, 0x53, 0x6b, 0x0d, 0xb2, 0x01, 0x05, 0x55, 0x85, 0x45, 0x61, 0xa9, 0xb8, 0xfa, + 0xe6, 0x84, 0x52, 0xfe, 0xbe, 0xcd, 0x1c, 0xc3, 0xd8, 0x1a, 0xc4, 0x37, 0x3c, 0x6b, 0x18, 0xb8, + 0x1e, 0x97, 0x1c, 0xb2, 0xe2, 0xeb, 0x50, 0xb0, 0x1c, 0x93, 0x1c, 0x6a, 0x96, 0x79, 0x58, 0x4d, + 0x2d, 0x0a, 0x4b, 0x65, 0x8e, 0xcf, 0x33, 0x70, 0xcb, 0x3c, 0xc4, 0x0b, 0x30, 0xeb, 0x91, 0x47, + 0xc4, 0xf3, 0x09, 0x53, 0x2d, 0x1f, 0xa9, 0xc6, 0x81, 0x58, 0x82, 0x2c, 0xf5, 0xaf, 0x5f, 0xcd, + 0x30, 0xdf, 0xdc, 0x9c, 0xee, 0x9b, 0x31, 0x03, 0x74, 0x27, 0xd2, 0x84, 0x71, 0xe3, 0xd7, 0x01, + 0x6c, 0x6b, 0x60, 0x05, 0xda, 0x9e, 0xe5, 0x04, 0xd5, 0xec, 0xa2, 0xb0, 0x94, 0xe6, 0x04, 0x05, + 0x06, 0xbf, 0x63, 0x39, 0x01, 0xf5, 0x93, 0xe5, 0x6b, 0xc6, 0x1e, 0x31, 0x1e, 0x56, 0x73, 0x49, + 0x65, 0x2c, 0xbf, 0x4e, 0x81, 0x58, 0x06, 0x78, 0x64, 0xf9, 0xd6, 0x8e, 0x65, 0x5b, 0xc1, 0x51, + 0x75, 0x76, 0x51, 0x58, 0xaa, 0xac, 0x2e, 0x4d, 0xd7, 0xa8, 0x63, 0xe8, 0xce, 0x76, 0x4c, 0xcf, + 0x85, 0x25, 0x24, 0xe0, 0x0f, 0xe0, 0xea, 0x40, 0x3f, 0xd4, 0x02, 0x6b, 0x40, 0xfc, 0x40, 0x1f, + 0x0c, 0x35, 0xbd, 0x4f, 0x34, 0x47, 0x77, 0x5c, 0xbf, 0x5a, 0x48, 0xc4, 0xe9, 0xca, 0x40, 0x3f, + 0xec, 0x46, 0x34, 0x62, 0x9f, 0xc8, 0x94, 0x02, 0x7f, 0x02, 0x88, 0xe7, 0xbd, 0xe6, 0x07, 0x1e, + 0x71, 0xfa, 0xc1, 0x5e, 0x15, 0x98, 0x4a, 0xb7, 0xa6, 0xc4, 0x8a, 0xea, 0xd3, 0x0e, 0x59, 0x3a, + 0x9c, 0x83, 0xef, 0x30, 0x67, 0x8f, 0x83, 0xf1, 0x0e, 0x5c, 0x8e, 0x84, 0x1f, 0xe8, 0x56, 0xa0, + 0x0d, 0x5d, 0xdb, 0x32, 0x8e, 0xaa, 0x45, 0x26, 0xff, 0xf6, 0xd3, 0xe5, 0xdf, 0xd7, 0xad, 0x60, + 0x8b, 0xf1, 0xf0, 0x1d, 0x2e, 0xd9, 0x93, 0x08, 0xfc, 0x26, 0x14, 0x87, 0xba, 0xa7, 0xdb, 0x36, + 0xb1, 0xad, 0xcf, 0x49, 0xb5, 0x94, 0xf0, 0x78, 0x12, 0x81, 0x65, 0xa8, 0xf8, 0x47, 0x7e, 0x40, + 0x06, 0x5a, 0x54, 0x27, 0xe5, 0xc5, 0xf4, 0x52, 0x65, 0xf5, 0xc6, 0x34, 0x35, 0x18, 0x71, 0x9d, + 0xd1, 0xde, 0xb5, 0x1c, 0x53, 0x2d, 0xfb, 0x09, 0x88, 0x8f, 0x25, 0x78, 0x8d, 0xa5, 0xa7, 0x66, + 0xf9, 0xda, 0xc8, 0x31, 0xdc, 0xc1, 0xc0, 0x0a, 0x02, 0x62, 0x6a, 0x34, 0xe1, 0x2c, 0xd7, 0xa9, + 0x56, 0x12, 0x8a, 0xbc, 0xc2, 0x28, 0x5b, 0x7e, 0xef, 0x98, 0x6e, 0x3b, 0x24, 0xdb, 0xc8, 0xe4, + 0xf3, 0xa8, 0x50, 0xfb, 0x4b, 0x06, 0xaa, 0x2d, 0x9a, 0xcc, 0x9d, 0x87, 0xd6, 0xf0, 0x05, 0xd5, + 0x50, 0x5c, 0x23, 0xe9, 0xe7, 0xaa, 0x91, 0xf1, 0xec, 0xce, 0x3c, 0x77, 0x76, 0x27, 0x4a, 0x3b, + 0x7b, 0x52, 0x69, 0x9f, 0x94, 0xc0, 0xb9, 0xef, 0x38, 0x81, 0x67, 0x2f, 0x32, 0x81, 0x9f, 0x96, + 0x48, 0xf9, 0x33, 0x25, 0x52, 0xed, 0xcb, 0x1c, 0x54, 0x36, 0x5c, 0xcb, 0xf9, 0xfe, 0x13, 0xe7, + 0x26, 0x54, 0x6c, 0xd7, 0x7d, 0x38, 0x1a, 0xc6, 0x95, 0x45, 0x33, 0xa8, 0xbc, 0x96, 0x42, 0x82, + 0x5a, 0x0e, 0x31, 0x51, 0xd1, 0xd4, 0x61, 0xd6, 0x0d, 0x4f, 0x39, 0x96, 0x19, 0xc5, 0xd5, 0x37, + 0xa6, 0x67, 0x86, 0x14, 0x9f, 0x85, 0x7c, 0xc7, 0x9c, 0xeb, 0x50, 0x18, 0x7e, 0x0f, 0x32, 0xf4, + 0x44, 0xe6, 0x51, 0xbe, 0x36, 0xc5, 0x2a, 0xea, 0x8b, 0xee, 0xd1, 0x90, 0x70, 0x66, 0xc6, 0x72, + 0xe1, 0xad, 0xf7, 0x3d, 0x78, 0x79, 0xdc, 0x74, 0x4d, 0xf7, 0x88, 0xf6, 0x90, 0x1c, 0x8d, 0x45, + 0xed, 0xf2, 0x98, 0x13, 0x44, 0x8f, 0xdc, 0x25, 0x47, 0x27, 0xe6, 0x6d, 0xe1, 0x3b, 0xce, 0x5b, + 0xb8, 0xc8, 0xbc, 0x7d, 0x07, 0x2e, 0x0d, 0x74, 0xcb, 0x09, 0x74, 0xcb, 0xd1, 0x5c, 0xcf, 0x24, + 0x9e, 0xe5, 0xf4, 0x59, 0x6b, 0x8f, 0xcc, 0x46, 0x11, 0x5a, 0xe1, 0xd8, 0x13, 0x7a, 0x70, 0xe9, + 0x3b, 0xed, 0xc1, 0xe5, 0xb3, 0xf6, 0xe0, 0x2c, 0xca, 0xd1, 0xa1, 0x0a, 0x3a, 0xae, 0x17, 0xf0, + 0xe2, 0xb9, 0x07, 0x73, 0xee, 0x28, 0x18, 0x8e, 0x82, 0x63, 0xe3, 0xc2, 0x32, 0xaa, 0x4d, 0xcf, + 0x97, 0xc8, 0x50, 0xbe, 0x65, 0x25, 0x14, 0x10, 0x9b, 0xbf, 0x0a, 0x38, 0x92, 0xa5, 0x0d, 0xf4, + 0xc0, 0xd8, 0xd3, 0x6c, 0xe2, 0x8c, 0x15, 0x15, 0x8a, 0xf0, 0x9b, 0x14, 0xdd, 0x26, 0x4e, 0xed, + 0x4b, 0x01, 0x4a, 0x0d, 0xcb, 0x0f, 0x2c, 0xc7, 0x08, 0x98, 0x5e, 0x37, 0x60, 0x8e, 0x11, 0x11, + 0x53, 0x4b, 0x0e, 0x7c, 0x65, 0xb5, 0xc2, 0xc1, 0x91, 0x73, 0x6e, 0x02, 0x32, 0x39, 0x63, 0x4c, + 0x99, 0x62, 0x94, 0x73, 0x11, 0x3c, 0x22, 0x5d, 0x05, 0xec, 0x8c, 0x6c, 0x3b, 0xcc, 0xde, 0x08, + 0x39, 0x36, 0x49, 0x21, 0x86, 0x17, 0x3d, 0x12, 0xe9, 0x82, 0xdf, 0x84, 0x12, 0xf1, 0x3c, 0xd7, + 0xd3, 0x5c, 0x47, 0x33, 0x47, 0x43, 0x56, 0xcf, 0x85, 0xa8, 0x44, 0x18, 0x46, 0x71, 0x1a, 0xa3, + 0x61, 0x0d, 0x41, 0x45, 0xf1, 0x4c, 0xcb, 0xd1, 0x69, 0xc1, 0x50, 0x0b, 0x6a, 0xff, 0x4c, 0x03, + 0xfa, 0xd8, 0xea, 0x7f, 0xae, 0xf7, 0x69, 0x8d, 0x72, 0x77, 0x37, 0x20, 0xc7, 0x02, 0x14, 0x8d, + 0xaf, 0xe7, 0x6b, 0x56, 0x9c, 0x17, 0x37, 0x01, 0xc8, 0xfe, 0x98, 0xb5, 0xc5, 0xd5, 0xeb, 0xd3, + 0xe3, 0xc5, 0xed, 0x8f, 0x66, 0x38, 0xb2, 0x7f, 0xec, 0xbb, 0x4a, 0xd8, 0xf5, 0xdc, 0x50, 0xf5, + 0xb1, 0x96, 0xc6, 0x30, 0xdc, 0xa6, 0x0b, 0x6a, 0x69, 0x77, 0xa1, 0xb4, 0x6b, 0x1d, 0xd2, 0xcc, + 0x65, 0x53, 0x7d, 0x35, 0xcb, 0x34, 0x3f, 0xa5, 0x33, 0x8d, 0x4f, 0xff, 0x6a, 0x91, 0x71, 0x87, + 0xc0, 0xe7, 0xe9, 0x8f, 0x3f, 0x7f, 0x4a, 0x41, 0xcd, 0x2e, 0xa6, 0x97, 0xf2, 0xa7, 0x9d, 0x42, + 0x7f, 0x4f, 0xc3, 0xdc, 0x26, 0xf1, 0xfa, 0x24, 0x11, 0xda, 0x4d, 0x28, 0xdb, 0x64, 0xf7, 0x39, + 0xea, 0xa8, 0x44, 0xd9, 0xe3, 0x2a, 0x52, 0xa0, 0xe2, 0x59, 0xfd, 0xbd, 0x84, 0xbc, 0xd4, 0x39, + 0xe5, 0x95, 0x19, 0x7f, 0x2c, 0x30, 0x11, 0xc1, 0xec, 0x8b, 0x38, 0x94, 0x6e, 0x42, 0x99, 0x56, + 0x97, 0x46, 0xf6, 0x47, 0x7a, 0x7c, 0x2e, 0x45, 0x85, 0x57, 0xa2, 0x28, 0x89, 0x63, 0xf0, 0xfb, + 0x70, 0x95, 0xb9, 0xf2, 0x38, 0xc9, 0xa7, 0x1c, 0x38, 0x64, 0x37, 0x90, 0xf6, 0xc7, 0x0f, 0x9c, + 0x9f, 0x41, 0x35, 0xf4, 0xdb, 0x09, 0xcc, 0x85, 0x04, 0xf3, 0x15, 0x46, 0x35, 0xc1, 0x5d, 0xfb, + 0x4d, 0x1a, 0x2a, 0x77, 0x74, 0x7f, 0x2f, 0x11, 0xd7, 0x5b, 0x30, 0x37, 0xa1, 0x4c, 0xd8, 0x89, + 0xf8, 0xc1, 0x9f, 0x54, 0x01, 0xdf, 0x06, 0x34, 0xb9, 0x79, 0xd8, 0x8c, 0x18, 0x71, 0x65, 0x7c, + 0xcb, 0x17, 0x1e, 0x91, 0xb7, 0xa0, 0x32, 0xa0, 0x49, 0x7c, 0xdc, 0x62, 0x93, 0x21, 0x29, 0x87, + 0xb8, 0x48, 0xd9, 0x17, 0x17, 0x93, 0xbf, 0xa5, 0x01, 0xb7, 0xf8, 0x1b, 0x42, 0x22, 0x2e, 0xdf, + 0xd3, 0xd8, 0xa7, 0x40, 0x39, 0x7a, 0xc0, 0x78, 0xd6, 0xf6, 0x57, 0x8a, 0x04, 0xb0, 0x80, 0xbd, + 0xe8, 0xa8, 0x9f, 0x38, 0xd0, 0xcc, 0x9e, 0x3a, 0xd0, 0x5c, 0xcc, 0xec, 0xbe, 0x91, 0xc9, 0xa7, + 0x51, 0xa6, 0xf6, 0x07, 0x01, 0xae, 0x44, 0xe1, 0x6c, 0x5a, 0x76, 0x40, 0x3c, 0x1e, 0xd0, 0x65, + 0x40, 0xb1, 0xa7, 0x0d, 0xd7, 0x66, 0x31, 0x11, 0x12, 0x31, 0xa9, 0x44, 0xd8, 0xba, 0x6b, 0xd3, + 0xc8, 0x7c, 0x3a, 0x19, 0x99, 0xb0, 0x41, 0xfe, 0x78, 0xc2, 0x19, 0xee, 0x30, 0x58, 0x4e, 0x3e, + 0x41, 0x2d, 0xd3, 0xcb, 0xdc, 0xb1, 0x63, 0xb7, 0x3c, 0x37, 0x70, 0x4f, 0x0a, 0x54, 0xed, 0xcf, + 0x79, 0xa8, 0x88, 0xfd, 0xbe, 0x47, 0xfa, 0x7a, 0xe0, 0x86, 0x2a, 0x5e, 0x07, 0xe8, 0x7b, 0x6e, + 0x38, 0x07, 0x27, 0x2b, 0xbb, 0xc0, 0xa0, 0x75, 0xd7, 0xf6, 0xf1, 0xa7, 0x50, 0xd2, 0x39, 0x93, + 0xe5, 0xc6, 0xd7, 0xcc, 0x9f, 0x4c, 0x8f, 0xf1, 0xf8, 0x16, 0xf1, 0x32, 0x91, 0x3e, 0x49, 0x79, + 0xf8, 0x47, 0x7c, 0xba, 0x22, 0xa6, 0x96, 0x50, 0x25, 0x13, 0xab, 0x82, 0x38, 0x76, 0x3d, 0xd6, + 0x68, 0x9d, 0xe7, 0x4a, 0x96, 0xe5, 0xca, 0xdb, 0x67, 0xd6, 0x64, 0x32, 0x73, 0xe6, 0x7f, 0x95, + 0x82, 0x62, 0x42, 0x3d, 0x2a, 0x78, 0x77, 0xe4, 0x18, 0x2c, 0x48, 0xe7, 0x11, 0xdc, 0x1c, 0x39, + 0x46, 0x24, 0x98, 0x0a, 0xc0, 0x8b, 0x90, 0x8f, 0xc7, 0xb1, 0x54, 0x22, 0x97, 0x62, 0x28, 0x7e, + 0x03, 0x2a, 0xbb, 0x2c, 0x57, 0xe2, 0xcc, 0xa0, 0x65, 0x58, 0x56, 0x4b, 0x21, 0x94, 0x67, 0xc4, + 0x55, 0xf6, 0x3a, 0xc8, 0xd0, 0x59, 0x36, 0x02, 0xe6, 0x8c, 0x10, 0x71, 0x07, 0x0a, 0xba, 0xd7, + 0x1f, 0x0d, 0x88, 0x13, 0xf8, 0xd5, 0x1c, 0x8b, 0xc8, 0x79, 0xaa, 0xee, 0x98, 0x99, 0xe7, 0xf0, + 0x7f, 0xd2, 0x90, 0xa1, 0x56, 0x60, 0x04, 0x25, 0x51, 0xfe, 0x48, 0x93, 0x95, 0xae, 0x26, 0xf7, + 0xda, 0x6d, 0x34, 0x83, 0x67, 0x21, 0x2d, 0x6e, 0xaf, 0x23, 0x01, 0x97, 0x20, 0xbf, 0xa6, 0x28, + 0x6d, 0x4d, 0x94, 0x1b, 0x28, 0x85, 0x8b, 0x30, 0xcb, 0x56, 0x8a, 0x8a, 0xd2, 0xb8, 0x02, 0x50, + 0x57, 0xe4, 0xba, 0xd8, 0xd5, 0xc4, 0xf5, 0x75, 0x94, 0xc1, 0x05, 0xc8, 0xd6, 0x95, 0x9e, 0xdc, + 0x45, 0x59, 0xca, 0xbe, 0x29, 0x3e, 0x40, 0xb3, 0xec, 0x47, 0x4b, 0x46, 0x79, 0x0c, 0x90, 0xeb, + 0x74, 0x1b, 0x0d, 0x69, 0x1b, 0x15, 0x28, 0xb0, 0xd3, 0xdb, 0x44, 0x40, 0xc5, 0x75, 0x7a, 0x9b, + 0x5a, 0x4b, 0xee, 0xa2, 0x22, 0xdd, 0x69, 0x5b, 0x54, 0x5b, 0xa2, 0x5c, 0x97, 0x50, 0x89, 0xa2, + 0x1e, 0x28, 0x2a, 0x93, 0x5c, 0x0e, 0x77, 0xea, 0xc9, 0x5d, 0x4d, 0x55, 0xee, 0x77, 0x50, 0x85, + 0xf1, 0xdd, 0x53, 0x1b, 0xad, 0x66, 0x13, 0xcd, 0x61, 0x0c, 0x95, 0x66, 0x4b, 0x16, 0xdb, 0x5a, + 0xcc, 0x8d, 0xa8, 0x41, 0x21, 0x8c, 0xef, 0x79, 0x09, 0x97, 0xa1, 0x20, 0xaa, 0xaa, 0xf8, 0x11, + 0x93, 0x88, 0xe9, 0x66, 0x1b, 0x1d, 0x45, 0x66, 0xab, 0xcb, 0x14, 0x49, 0x57, 0x6b, 0x6c, 0x79, + 0x85, 0x6e, 0xd7, 0xe9, 0xaa, 0x2d, 0x79, 0x9d, 0xad, 0x5f, 0x62, 0x56, 0xb7, 0xba, 0xcc, 0x05, + 0x2f, 0x53, 0x43, 0xe8, 0x42, 0x51, 0xd1, 0x55, 0x9c, 0x87, 0x4c, 0x5d, 0x51, 0x55, 0x54, 0xc5, + 0x55, 0xb8, 0xb2, 0x25, 0xa9, 0x75, 0x49, 0xee, 0xb6, 0xda, 0x92, 0xd6, 0x68, 0x75, 0xea, 0x5a, + 0x6b, 0x73, 0xab, 0x8d, 0x5e, 0x99, 0xc0, 0xd4, 0x15, 0xb9, 0x1b, 0x62, 0xe6, 0xf1, 0x65, 0x98, + 0x63, 0x3a, 0x28, 0x6b, 0x1b, 0x52, 0x3d, 0x74, 0xe2, 0xab, 0xf8, 0x0a, 0xa0, 0x50, 0x95, 0x04, + 0xf4, 0x35, 0xaa, 0xc1, 0xb6, 0xa8, 0x6a, 0x5b, 0xca, 0x16, 0xfa, 0x41, 0xa8, 0x1e, 0x35, 0x8b, + 0xad, 0x17, 0xf0, 0x1c, 0x14, 0x3b, 0x5d, 0x6d, 0x53, 0xbc, 0x2b, 0xb5, 0x5b, 0xb2, 0x84, 0xae, + 0x51, 0x73, 0x3a, 0x5d, 0x4d, 0x7a, 0xd0, 0x95, 0xe4, 0x2e, 0x5a, 0xa4, 0xb6, 0x76, 0xba, 0x5a, + 0x4f, 0x6e, 0x29, 0x32, 0xba, 0x5e, 0xbb, 0x0d, 0x19, 0x5a, 0x12, 0x54, 0x77, 0xb1, 0xd7, 0x55, + 0xd0, 0x0c, 0x0b, 0x4d, 0x5d, 0x6c, 0x8b, 0x2a, 0x12, 0xa8, 0x6c, 0x59, 0x91, 0x35, 0xbe, 0x4e, + 0xd5, 0x7e, 0x39, 0x0b, 0xaf, 0xb6, 0x9c, 0x80, 0x78, 0x36, 0xd1, 0x1f, 0x11, 0x33, 0x7c, 0xaf, + 0x48, 0x1c, 0x5f, 0x1f, 0x4d, 0xdc, 0x04, 0x3e, 0x98, 0x9e, 0x8f, 0xa7, 0x88, 0x09, 0x8f, 0xb7, + 0x89, 0xeb, 0x41, 0xe2, 0x2d, 0x29, 0x75, 0xd2, 0x5b, 0xd2, 0xf8, 0xfb, 0x6e, 0xfa, 0xe4, 0xf7, + 0xdd, 0x0b, 0x7d, 0xc3, 0xc8, 0x9e, 0xff, 0x98, 0xfa, 0x7f, 0x7f, 0xf0, 0x9a, 0xff, 0x45, 0x1a, + 0xb2, 0x2c, 0x3a, 0xf8, 0x43, 0xc8, 0x98, 0xc4, 0x37, 0x9e, 0x69, 0x50, 0x61, 0x9c, 0x67, 0x99, + 0x53, 0xea, 0x90, 0x19, 0xba, 0x7e, 0x18, 0xce, 0x53, 0x9f, 0x35, 0xb7, 0x5c, 0x3f, 0xd8, 0x0a, + 0xbf, 0xd8, 0xd0, 0x0c, 0x8a, 0xf6, 0xa1, 0xcc, 0xb8, 0x01, 0xf9, 0x78, 0x24, 0xc8, 0x9c, 0xf3, + 0xba, 0x11, 0x73, 0x1e, 0x3f, 0xb1, 0x66, 0x9f, 0xeb, 0x89, 0xf5, 0x69, 0x53, 0x47, 0xee, 0x6c, + 0x2f, 0x86, 0x7f, 0x15, 0xa0, 0xb2, 0xe5, 0xb9, 0x9f, 0x11, 0x23, 0xe8, 0x90, 0xf0, 0x71, 0xe1, + 0x43, 0xc8, 0xd2, 0xc4, 0x8e, 0x4a, 0xef, 0x3c, 0x99, 0x1d, 0x32, 0xe2, 0x75, 0xb8, 0xd4, 0x27, + 0x0e, 0xf1, 0xf4, 0x20, 0x31, 0x3d, 0x87, 0x17, 0xf1, 0xf9, 0xc9, 0xf8, 0x92, 0xc1, 0x72, 0xf8, + 0x69, 0xad, 0xab, 0xa2, 0x98, 0x29, 0x1a, 0xab, 0x6f, 0x00, 0x72, 0x46, 0xec, 0xa1, 0xc8, 0xd7, + 0x86, 0xc4, 0xd3, 0xfa, 0xc4, 0x09, 0x2f, 0xe1, 0x6a, 0xd9, 0x19, 0x0d, 0xe8, 0xf9, 0xbd, 0x45, + 0xbc, 0x75, 0xe2, 0xd4, 0xbe, 0x29, 0x43, 0xe9, 0xbe, 0xe5, 0x98, 0xee, 0x01, 0x6f, 0x20, 0x8b, + 0xec, 0x8b, 0x40, 0x60, 0xb1, 0x49, 0xe1, 0x88, 0xbf, 0x8e, 0x24, 0x41, 0xb8, 0x03, 0x85, 0x03, + 0xc6, 0xd1, 0x8c, 0x95, 0x5b, 0x99, 0x6e, 0x6a, 0x52, 0x38, 0x5f, 0x34, 0xe3, 0x03, 0x30, 0x96, + 0x33, 0xff, 0x47, 0x81, 0x1f, 0x7d, 0x1d, 0x28, 0x47, 0x83, 0x09, 0x69, 0x3e, 0xeb, 0x18, 0xa0, + 0x8e, 0xcb, 0xc0, 0xf7, 0x00, 0xf8, 0x56, 0x54, 0x62, 0x8a, 0x49, 0x7c, 0xe7, 0x7c, 0x3a, 0x53, + 0xa9, 0x09, 0x21, 0xef, 0x67, 0x1e, 0x7f, 0x71, 0x4d, 0x98, 0xff, 0x62, 0x16, 0xb2, 0x4d, 0x4f, + 0x1f, 0x10, 0x7c, 0x17, 0x32, 0x03, 0xd7, 0x24, 0x5c, 0xdd, 0xb3, 0x0a, 0x67, 0xbc, 0xcb, 0x9b, + 0xae, 0x19, 0x77, 0x29, 0x2a, 0x04, 0xdf, 0x83, 0xdc, 0x8e, 0x3b, 0x72, 0x4c, 0x7f, 0xca, 0xf0, + 0x79, 0xba, 0xb8, 0x35, 0xc6, 0x1a, 0xf5, 0xcc, 0x50, 0x10, 0xfe, 0x18, 0x0a, 0xe4, 0xd0, 0xb0, + 0x47, 0x2c, 0xc7, 0xd3, 0x4c, 0xc9, 0x77, 0xcf, 0x25, 0x55, 0x8a, 0xb8, 0xe3, 0x07, 0x9f, 0x08, + 0x30, 0xff, 0x5f, 0x01, 0xb2, 0x6c, 0x53, 0xba, 0x0b, 0xdb, 0x8f, 0xf6, 0x5d, 0xee, 0x8a, 0x77, + 0xcf, 0xaf, 0x7b, 0xa2, 0x6b, 0x1f, 0x8b, 0xa3, 0xe7, 0x8b, 0xe5, 0x04, 0x9a, 0xbb, 0xbb, 0xeb, + 0x93, 0x70, 0xa0, 0x8b, 0x3e, 0xce, 0x15, 0x2c, 0x27, 0x50, 0x18, 0x18, 0x5f, 0x87, 0x12, 0xad, + 0x0a, 0x33, 0x22, 0xa3, 0x96, 0x96, 0xd4, 0x22, 0x83, 0x71, 0x92, 0x0d, 0x28, 0x86, 0x48, 0xf6, + 0x49, 0x9b, 0x37, 0xa4, 0x73, 0x7c, 0xf0, 0x85, 0x90, 0x9b, 0xea, 0x34, 0xff, 0x3b, 0x01, 0x72, + 0xa1, 0xbb, 0xb1, 0x0c, 0x59, 0x3f, 0xd0, 0xbd, 0x80, 0xf7, 0xe3, 0xd5, 0xf3, 0x9b, 0x1d, 0xf7, + 0x29, 0x2a, 0x06, 0x37, 0x20, 0x4d, 0x1c, 0x93, 0x27, 0xc0, 0x33, 0x48, 0x53, 0x29, 0x7b, 0xed, + 0x06, 0x64, 0x68, 0x76, 0xd1, 0x59, 0x50, 0x15, 0xe5, 0x75, 0x09, 0xcd, 0xd0, 0x41, 0x83, 0x8d, + 0x6d, 0x02, 0x1d, 0x34, 0xd6, 0x55, 0xa5, 0xb7, 0xd5, 0x41, 0xa9, 0xda, 0xe7, 0x50, 0x88, 0x7d, + 0x8f, 0xaf, 0xc2, 0xe5, 0x9e, 0xbc, 0xa6, 0xf4, 0xe4, 0x86, 0xd4, 0xd0, 0xb6, 0x54, 0xa9, 0x2e, + 0x35, 0x5a, 0xf2, 0x3a, 0x9a, 0x19, 0x47, 0x34, 0x95, 0x76, 0x5b, 0xb9, 0x4f, 0x11, 0x02, 0x1d, + 0x93, 0x94, 0x66, 0xb3, 0x23, 0x75, 0x13, 0xe4, 0xa9, 0x04, 0xf4, 0x98, 0x36, 0x4d, 0xe7, 0xa3, + 0x7a, 0x4f, 0x55, 0xa5, 0x70, 0x7e, 0x44, 0x99, 0xda, 0x27, 0x50, 0x88, 0xb3, 0x8b, 0x8e, 0x8a, + 0xb2, 0xa2, 0x49, 0x0f, 0xea, 0xed, 0x5e, 0x87, 0x4e, 0x48, 0x6c, 0x53, 0xb6, 0x6c, 0x48, 0x5a, + 0x92, 0x4f, 0xc0, 0x97, 0xa0, 0x1c, 0x21, 0x98, 0x1d, 0x28, 0x45, 0xb9, 0x23, 0x50, 0xb7, 0x25, + 0x75, 0x50, 0x7a, 0xfe, 0x1f, 0x29, 0xc8, 0x47, 0x7d, 0x07, 0x4b, 0x89, 0xbb, 0x45, 0x71, 0xf5, + 0xad, 0xb3, 0x7a, 0x75, 0xf2, 0x66, 0x71, 0x31, 0x07, 0xda, 0x1a, 0x64, 0x77, 0x69, 0xbc, 0xf8, + 0x85, 0xfd, 0xf6, 0x79, 0x62, 0xac, 0x86, 0xac, 0x78, 0x09, 0xc6, 0xee, 0x2a, 0xec, 0xf4, 0xca, + 0x46, 0x37, 0xbc, 0xb1, 0x5b, 0xcc, 0x3c, 0xe4, 0x75, 0xaf, 0xef, 0xb7, 0xcc, 0x43, 0x9f, 0xbd, + 0x44, 0x96, 0xd5, 0x78, 0x4d, 0xa5, 0x84, 0xaf, 0xed, 0x5c, 0x4a, 0x3e, 0x31, 0x0b, 0x8c, 0x61, + 0x36, 0x32, 0xf9, 0x14, 0x4a, 0xf3, 0xeb, 0xca, 0xef, 0x05, 0x80, 0xe3, 0xee, 0x48, 0x47, 0x55, + 0x55, 0xb9, 0xaf, 0xc9, 0xbd, 0xcd, 0x35, 0x49, 0xe5, 0x79, 0x26, 0xca, 0x77, 0xc3, 0x21, 0xb6, + 0x21, 0xc9, 0x1d, 0x49, 0x63, 0x6b, 0x16, 0x24, 0x3e, 0x82, 0x87, 0x90, 0x34, 0x9d, 0x90, 0xeb, + 0xbd, 0x4d, 0x36, 0xa8, 0x77, 0xc3, 0x9b, 0x0b, 0x1b, 0xcf, 0xc3, 0x9b, 0x4b, 0x5b, 0x5c, 0x47, + 0x39, 0x2a, 0xae, 0x2d, 0x89, 0x0d, 0x34, 0x4b, 0xf3, 0xa7, 0xd9, 0x52, 0x3b, 0x5d, 0x6d, 0x5b, + 0x6c, 0xf7, 0x24, 0x94, 0xa7, 0xf2, 0xdb, 0x62, 0xbc, 0x2e, 0x50, 0x69, 0x72, 0xf7, 0x0e, 0x5f, + 0xc2, 0xad, 0x9f, 0x42, 0x65, 0xfc, 0x5b, 0x14, 0x4d, 0xfc, 0xad, 0xde, 0x5a, 0xbb, 0x55, 0x47, + 0x33, 0xf8, 0x15, 0x78, 0x29, 0xfc, 0x4d, 0xef, 0x13, 0xec, 0xca, 0xc5, 0x51, 0xc2, 0xda, 0xdb, + 0x8f, 0xbf, 0x59, 0x98, 0x79, 0xfc, 0x64, 0x41, 0xf8, 0xea, 0xc9, 0x82, 0xf0, 0xf5, 0x93, 0x05, + 0xe1, 0xdf, 0x4f, 0x16, 0x84, 0x5f, 0x7f, 0xbb, 0x30, 0xf3, 0xd5, 0xb7, 0x0b, 0x33, 0x5f, 0x7f, + 0xbb, 0x30, 0xf3, 0x71, 0x31, 0xf1, 0x0f, 0x2b, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xc9, 0xbd, + 0x22, 0xdc, 0xa0, 0x23, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 71e427fb4e3b..e0d74da628fe 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -64,6 +64,7 @@ enum ScanVisibility { // - JSON of all decoded column values. message TableReaderSpec { optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; + optional bool table_is_uncommitted_version = 14 [(gogoproto.nullable) = false]; // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. table.indexes[index_idx-1] optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; @@ -148,6 +149,7 @@ message TableReaderSpec { // when operating on a prefix of a compound key. message IndexSkipTableReaderSpec { optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; + optional bool table_is_uncommitted_version = 8 [(gogoproto.nullable) = false]; // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. table.indexes[index_idx-1] optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; @@ -196,6 +198,7 @@ message IndexSkipTableReaderSpec { // Internal columns: | c | d | e | message JoinReaderSpec { optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; + optional bool table_is_uncommitted_version = 13 [(gogoproto.nullable) = false]; // If 0, we use the primary index; each row in the input stream has a value // for each primary key. The index must provide all lookup columns. @@ -332,6 +335,7 @@ message ZigzagJoinerSpec { // TODO(pbardea): Replace these with inputs that conform to a RowSource-like // interface. repeated sqlbase.TableDescriptor tables = 1 [(gogoproto.nullable) = false]; + repeated bool table_is_uncommitted_version = 7; // An array of arrays. The array at eq_columns[side_idx] contains the // equality columns for that side. All arrays in eq_columns should have @@ -500,6 +504,7 @@ message HashJoinerSpec { // Internal columns for LEFT_SEMI and LEFT_ANTI: | a | b | message InvertedJoinerSpec { optional sqlbase.TableDescriptor table = 1 [(gogoproto.nullable) = false]; + optional bool table_is_uncommitted_version = 8 [(gogoproto.nullable) = false]; // The ID of the inverted index. The first column in the index is the // inverted column, and the the remaining columns are the primary key. @@ -689,6 +694,7 @@ message AggregatorSpec { message InterleavedReaderJoinerSpec { message Table { optional sqlbase.TableDescriptor desc = 1 [(gogoproto.nullable) = false]; + optional bool table_is_uncommitted_version = 6 [(gogoproto.nullable) = false]; // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. desc.indexes[index_idx-1] optional uint32 index_idx = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index e5d5208b6e24..0404456f534d 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -15,6 +15,7 @@ import ( "sort" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -24,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowcontainer" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/scrub" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/span" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" @@ -72,7 +74,7 @@ type joinReader struct { diskMonitor *mon.BytesMonitor - desc tabledesc.Immutable + desc catalog.TableDescriptor index *descpb.IndexDescriptor colIdxMap map[descpb.ColumnID]int maintainOrdering bool @@ -137,8 +139,22 @@ func newJoinReader( default: return nil, errors.Errorf("unsupported joinReaderType") } + var table catalog.TableDescriptor + if spec.TableIsUncommittedVersion { + table = tabledesc.NewImmutable(spec.Table) + } else { + var err error + table, err = flowCtx.TypeResolverFactory.Descriptors.GetTableVersionByIDWithMinVersion( + flowCtx.EvalCtx.Ctx(), spec.Table.ModificationTime, + flowCtx.EvalCtx.Txn, + spec.Table.ID, + spec.Table.Version, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return nil, err + } + } jr := &joinReader{ - desc: tabledesc.MakeImmutable(spec.Table), + desc: table, maintainOrdering: spec.MaintainOrdering, input: input, inputTypes: input.OutputTypes(), @@ -235,7 +251,7 @@ func newJoinReader( } _, _, err = initRowFetcher( - flowCtx, &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ + flowCtx, &fetcher, jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ rightCols, false /* isCheck */, jr.EvalCtx.Mon, &jr.alloc, spec.Visibility, spec.LockingStrength, spec.LockingWaitPolicy, sysColDescs, ) @@ -265,7 +281,7 @@ func (jr *joinReader) initJoinReaderStrategy( neededRightCols util.FastIntSet, readerType joinReaderType, ) { - spanBuilder := span.MakeBuilder(flowCtx.Codec(), &jr.desc, jr.index) + spanBuilder := span.MakeBuilder(flowCtx.Codec(), jr.desc, jr.index) spanBuilder.SetNeededColumns(neededRightCols) var keyToInputRowIndices map[string][]int diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index 2b213009b574..d2693e0134ee 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" @@ -61,7 +60,7 @@ type rowFetcher interface { func initRowFetcher( flowCtx *execinfra.FlowCtx, fetcher *row.Fetcher, - desc *tabledesc.Immutable, + desc catalog.TableDescriptor, indexIdx int, colIdxMap map[descpb.ColumnID]int, reverseScan bool, @@ -79,9 +78,9 @@ func initRowFetcher( return nil, false, err } - cols := desc.Columns + cols := desc.GetPublicColumns() if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { - cols = desc.ReadableColumns + cols = desc.GetReadableColumns() } // Add on any requested system columns. We slice cols to avoid modifying // the underlying table descriptor. diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 42138f3e3d7d..dd112e4f8843 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -17,12 +17,14 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -89,8 +91,21 @@ func newTableReader( // just in case. tr.parallelize = spec.Parallelize && tr.limitHint == 0 tr.maxTimestampAge = time.Duration(spec.MaxTimestampAgeNanos) + var tableDesc catalog.TableDescriptor + if spec.TableIsUncommittedVersion { + tableDesc = tabledesc.NewImmutable(spec.Table) + } else { + var err error + tableDesc, err = flowCtx.TypeResolverFactory.Descriptors.GetTableVersionByIDWithMinVersion( + flowCtx.EvalCtx.Ctx(), spec.Table.ModificationTime, + flowCtx.EvalCtx.Txn, + spec.Table.ID, spec.Table.Version, + tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return nil, err + } + } - tableDesc := tabledesc.NewImmutable(spec.Table) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic resultTypes := tableDesc.ColumnTypesWithMutations(returnMutations) columnIdxMap := tableDesc.ColumnIdxMapWithMutations(returnMutations) diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 34814b388029..d785ee5b0439 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -280,9 +281,22 @@ func newZigzagJoiner( z := &zigzagJoiner{} // TODO(ajwerner): Utilize a cached copy of these tables. - tables := make([]tabledesc.Immutable, len(spec.Tables)) + tables := make([]catalog.TableDescriptor, len(spec.Tables)) for i := range spec.Tables { - tables[i] = tabledesc.MakeImmutable(spec.Tables[i]) + if !spec.TableIsUncommittedVersion[i] { + flags := tree.ObjectLookupFlagsWithRequired() + flags.DesiredObjectKind = tree.TableObject + table, err := flowCtx.TypeResolverFactory.Descriptors.GetTableVersionByIDWithMinVersion( + flowCtx.EvalCtx.Ctx(), spec.Tables[i].ModificationTime, + flowCtx.EvalCtx.Txn, + spec.Tables[i].GetID(), spec.Tables[i].Version, + tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return nil, err + } + tables[i] = table + } + tables[i] = tabledesc.NewImmutable(spec.Tables[i]) } leftColumnTypes := tables[0].ColumnTypes() rightColumnTypes := tables[1].ColumnTypes() @@ -338,7 +352,7 @@ func newZigzagJoiner( if err := z.setupInfo(flowCtx, spec, i, colOffset, tables); err != nil { return nil, err } - colOffset += len(z.infos[i].table.Columns) + colOffset += len(z.infos[i].table.GetPublicColumns()) } z.side = 0 return z, nil @@ -375,7 +389,7 @@ func (z *zigzagJoiner) Start(ctx context.Context) context.Context { type zigzagJoinerInfo struct { fetcher row.Fetcher alloc *rowenc.DatumAlloc - table *tabledesc.Immutable + table catalog.TableDescriptor index *descpb.IndexDescriptor indexTypes []*types.T indexDirs []descpb.IndexDescriptor_Direction @@ -412,19 +426,20 @@ func (z *zigzagJoiner) setupInfo( spec *execinfrapb.ZigzagJoinerSpec, side int, colOffset int, - tables []tabledesc.Immutable, + tables []catalog.TableDescriptor, ) error { z.side = side info := z.infos[side] info.alloc = &rowenc.DatumAlloc{} - info.table = &tables[side] + info.table = tables[side] info.eqColumns = spec.EqColumns[side].Columns indexOrdinal := spec.IndexOrdinals[side] if indexOrdinal == 0 { - info.index = &info.table.PrimaryIndex + info.index = info.table.GetPrimaryIndex() } else { - info.index = &info.table.Indexes[indexOrdinal-1] + indexes := info.table.GetPublicNonPrimaryIndexes() + info.index = &indexes[indexOrdinal-1] } var columnIDs []descpb.ColumnID @@ -439,7 +454,7 @@ func (z *zigzagJoiner) setupInfo( // Add the outputted columns. neededCols := util.MakeFastIntSet() outCols := z.Out.NeededColumns() - maxCol := colOffset + len(info.table.Columns) + maxCol := colOffset + len(info.table.GetPublicColumns()) for i, ok := outCols.Next(colOffset); ok && i < maxCol; i, ok = outCols.Next(i + 1) { neededCols.Add(i - colOffset) } @@ -662,7 +677,7 @@ func (zi *zigzagJoinerInfo) eqColTypes() []*types.T { func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { ordering := make(colinfo.ColumnOrdering, len(zi.eqColumns)) for i := range zi.eqColumns { - colID := zi.table.Columns[zi.eqColumns[i]].ID + colID := zi.table.GetColumnAtIdx(int(zi.eqColumns[i])).ID // Search the index columns, then the primary keys to find an ordering for // the current column, 'colID'. var direction encoding.Direction @@ -672,8 +687,8 @@ func (zi *zigzagJoinerInfo) eqOrdering() (colinfo.ColumnOrdering, error) { if err != nil { return nil, err } - } else if idx := findColumnID(zi.table.PrimaryIndex.ColumnIDs, colID); idx != -1 { - direction, err = zi.table.PrimaryIndex.ColumnDirections[idx].ToEncodingDirection() + } else if idx := findColumnID(zi.table.GetPrimaryIndex().ColumnIDs, colID); idx != -1 { + direction, err = zi.table.GetPrimaryIndex().ColumnDirections[idx].ToEncodingDirection() if err != nil { return nil, err } diff --git a/pkg/sql/span/span_builder.go b/pkg/sql/span/span_builder.go index 421cb0109948..100c8e4f8ff7 100644 --- a/pkg/sql/span/span_builder.go +++ b/pkg/sql/span/span_builder.go @@ -13,8 +13,8 @@ package span import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/opt/constraint" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -28,7 +28,7 @@ import ( // Builder is a single struct for generating key spans from Constraints, Datums and encDatums. type Builder struct { codec keys.SQLCodec - table *tabledesc.Immutable + table catalog.TableDescriptor index *descpb.IndexDescriptor indexColTypes []*types.T indexColDirs []descpb.IndexDescriptor_Direction @@ -52,7 +52,7 @@ var _ = (*Builder).UnsetNeededFamilies // MakeBuilder creates a Builder for a table and index. func MakeBuilder( - codec keys.SQLCodec, table *tabledesc.Immutable, index *descpb.IndexDescriptor, + codec keys.SQLCodec, table catalog.TableDescriptor, index *descpb.IndexDescriptor, ) *Builder { s := &Builder{ codec: codec, @@ -68,7 +68,8 @@ func MakeBuilder( s.indexColTypes = make([]*types.T, len(columnIDs)) for i, colID := range columnIDs { // TODO (rohany): do I need to look at table columns with mutations here as well? - for _, col := range table.Columns { + columns := table.GetPublicColumns() + for _, col := range columns { if col.ID == colID { s.indexColTypes[i] = col.Type break @@ -92,7 +93,7 @@ func MakeBuilder( s.interstices[sharedPrefixLen]) } s.interstices[sharedPrefixLen] = rowenc.EncodePartialTableIDIndexID( - s.interstices[sharedPrefixLen], table.ID, index.ID) + s.interstices[sharedPrefixLen], table.GetID(), index.ID) } return s @@ -183,15 +184,15 @@ func (s *Builder) CanSplitSpanIntoSeparateFamilies( // and it cannot be an inverted index. // * We have all of the lookup columns of the index. // * We don't need all of the families. - return s.index.Unique && len(s.table.Families) > 1 && - (s.index.ID == s.table.PrimaryIndex.ID || + return s.index.Unique && len(s.table.GetFamilies()) > 1 && + (s.index.ID == s.table.GetPrimaryIndexID() || // Secondary index specific checks. (s.index.Version == descpb.SecondaryIndexFamilyFormatVersion && !containsNull && len(s.index.StoreColumnIDs) > 0 && s.index.Type == descpb.IndexDescriptor_FORWARD)) && prefixLen == len(s.index.ColumnIDs) && - numNeededFamilies < len(s.table.Families) + numNeededFamilies < len(s.table.GetFamilies()) } // Functions for optimizer related span generation are below.