From 2b6f52b326e4f26d82c6fd26ba58c1b2368adb52 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Mon, 28 Dec 2020 10:29:10 -0800 Subject: [PATCH 01/10] opt: do not derive prune columns for Upsert, Update, Delete We no longer derive output prune columns for Upsert, Update, and Delete ops in `DerivePruneCols`. There are no PruneCols rules for these operators, so deriving their prune columns was only performing unnecessary work. There are other rules that prune the fetch and return columns for these operators. These rules do not rely on `DerivePruneCols`. Release note: None --- pkg/sql/opt/memo/testdata/logprops/delete | 3 --- pkg/sql/opt/norm/prune_cols_funcs.go | 22 +--------------------- 2 files changed, 1 insertion(+), 24 deletions(-) diff --git a/pkg/sql/opt/memo/testdata/logprops/delete b/pkg/sql/opt/memo/testdata/logprops/delete index a499a2b85e91..013da35d6772 100644 --- a/pkg/sql/opt/memo/testdata/logprops/delete +++ b/pkg/sql/opt/memo/testdata/logprops/delete @@ -64,7 +64,6 @@ project ├── volatile, mutations ├── key: (5) ├── fd: ()-->(1), (5)-->(2-4) - ├── prune: (1-4) └── select ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) ├── key: (12) @@ -107,7 +106,6 @@ project ├── volatile, mutations ├── key: () ├── fd: ()-->(1-5) - ├── prune: (1-4) └── select ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) ├── cardinality: [0 - 1] @@ -148,7 +146,6 @@ project ├── volatile, mutations ├── key: (5) ├── fd: (2)==(3), (3)==(2), (5)-->(1-4) - ├── prune: (1-4) └── select ├── columns: a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) ├── key: (12) diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index ca0cf56b424b..831f982ba586 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -89,17 +89,8 @@ func (c *CustomFuncs) NeededMutationCols( func (c *CustomFuncs) NeededMutationFetchCols( op opt.Operator, private *memo.MutationPrivate, ) opt.ColSet { - return neededMutationFetchCols(c.mem, op, private) -} - -// neededMutationFetchCols returns the set of columns needed by the given -// mutation operator. -func neededMutationFetchCols( - mem *memo.Memo, op opt.Operator, private *memo.MutationPrivate, -) opt.ColSet { - var cols opt.ColSet - tabMeta := mem.Metadata().TableMeta(private.Table) + tabMeta := c.mem.Metadata().TableMeta(private.Table) // familyCols returns the columns in the given family. familyCols := func(fam cat.Family) opt.ColSet { @@ -579,17 +570,6 @@ func DerivePruneCols(e memo.RelExpr) opt.ColSet { relProps.Rule.PruneCols.DifferenceWith(w.ScalarProps().OuterCols) } - case opt.UpdateOp, opt.UpsertOp, opt.DeleteOp: - // Find the columns that would need to be fetched, if no returning - // clause were present. - withoutReturningPrivate := *e.Private().(*memo.MutationPrivate) - withoutReturningPrivate.ReturnCols = opt.OptionalColList{} - neededCols := neededMutationFetchCols(e.Memo(), e.Op(), &withoutReturningPrivate) - - // Only the "free" RETURNING columns can be pruned away (i.e. the columns - // required by the mutation only because they're being returned). - relProps.Rule.PruneCols = relProps.OutputCols.Difference(neededCols) - case opt.WithOp: // WithOp passes through its input unchanged, so it has the same pruning // characteristics as its input. From 07e93642b95bcdcb7af4fc5442fd5b936e8699b5 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Mon, 28 Dec 2020 15:26:10 -0800 Subject: [PATCH 02/10] sql: remove logic to determine fetch cols in row updater Previously, the `row.MakeUpdater` function had logic to determine the fetch columns required for an update operation. This is not necessary because the cost based optimizer already determines the necessary fetch columns and plumbs them to `MakeUpdater` as the `requestedCols` argument. Release note: None --- pkg/sql/backfill/backfill.go | 3 +- pkg/sql/row/updater.go | 67 ++---------------------------------- 2 files changed, 5 insertions(+), 65 deletions(-) diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 6038384b545c..030cab6a678a 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -253,9 +253,10 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( ) (roachpb.Key, error) { // TODO(dan): Tighten up the bound on the requestedCols parameter to // makeRowUpdater. - requestedCols := make([]descpb.ColumnDescriptor, 0, len(tableDesc.Columns)+len(cb.added)) + requestedCols := make([]descpb.ColumnDescriptor, 0, len(tableDesc.Columns)+len(cb.added)+len(cb.dropped)) requestedCols = append(requestedCols, tableDesc.Columns...) requestedCols = append(requestedCols, cb.added...) + requestedCols = append(requestedCols, cb.dropped...) ru, err := row.MakeUpdater( ctx, txn, diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 065480a37ed3..ebb451dce3d9 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -149,9 +149,6 @@ func MakeUpdater( } } - // Columns of the table to update, including those in delete/write-only state - tableCols := tableDesc.DeletableColumns() - var deleteOnlyIndexes []descpb.IndexDescriptor for _, idx := range tableDesc.DeleteOnlyIndexes() { if needsUpdate(idx) { @@ -172,6 +169,8 @@ func MakeUpdater( ru := Updater{ Helper: newRowHelper(codec, tableDesc, includeIndexes), DeleteHelper: deleteOnlyHelper, + FetchCols: requestedCols, + FetchColIDtoRowIndex: ColIDtoRowIndexFromCols(requestedCols), UpdateCols: updateCols, UpdateColIDtoRowIndex: updateColIDtoRowIndex, primaryKeyColChange: primaryKeyColChange, @@ -185,73 +184,13 @@ func MakeUpdater( // When changing the primary key, we delete the old values and reinsert // them, so request them all. var err error + tableCols := tableDesc.DeletableColumns() ru.rd = MakeDeleter(codec, tableDesc, tableCols) - ru.FetchCols = ru.rd.FetchCols - ru.FetchColIDtoRowIndex = ColIDtoRowIndexFromCols(ru.FetchCols) if ru.ri, err = MakeInserter( ctx, txn, codec, tableDesc, tableCols, alloc, ); err != nil { return Updater{}, err } - } else { - ru.FetchCols = requestedCols[:len(requestedCols):len(requestedCols)] - ru.FetchColIDtoRowIndex = ColIDtoRowIndexFromCols(ru.FetchCols) - - // maybeAddCol adds the provided column to ru.FetchCols and - // ru.FetchColIDtoRowIndex if it isn't already present. - maybeAddCol := func(colID descpb.ColumnID) error { - if _, ok := ru.FetchColIDtoRowIndex.Get(colID); !ok { - col, _, err := tableDesc.FindReadableColumnByID(colID) - if err != nil { - return err - } - ru.FetchColIDtoRowIndex.Set(col.ID, len(ru.FetchCols)) - ru.FetchCols = append(ru.FetchCols, *col) - } - return nil - } - - // Fetch all columns in the primary key so that we can construct the - // keys when writing out the new kvs to the primary index. - for _, colID := range tableDesc.GetPrimaryIndex().ColumnIDs { - if err := maybeAddCol(colID); err != nil { - return Updater{}, err - } - } - - // If any part of a column family is being updated, fetch all columns in - // that column family so that we can reconstruct the column family with - // the updated columns before writing it. - for i := range tableDesc.Families { - family := &tableDesc.Families[i] - familyBeingUpdated := false - for _, colID := range family.ColumnIDs { - if _, ok := ru.UpdateColIDtoRowIndex.Get(colID); ok { - familyBeingUpdated = true - break - } - } - if familyBeingUpdated { - for _, colID := range family.ColumnIDs { - if err := maybeAddCol(colID); err != nil { - return Updater{}, err - } - } - } - } - - // Fetch all columns from indices that are being update so that they can - // be used to create the new kv pairs for those indices. - for _, index := range includeIndexes { - if err := index.RunOverAllColumns(maybeAddCol); err != nil { - return Updater{}, err - } - } - for _, index := range deleteOnlyIndexes { - if err := index.RunOverAllColumns(maybeAddCol); err != nil { - return Updater{}, err - } - } } // If we are fetching from specific families, we might get From c5e72abb2755dcb7a3556cc5743438540e303826 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 29 Dec 2020 16:57:43 -0800 Subject: [PATCH 03/10] opt: safer access to partial index predicates in TableMeta Previously, partial index predicate expressions in TableMeta were the source-of-truth used within the optimizer to determine if an index is a partial index. However, partial index predicates are not added to TableMeta for all types of statements in optbuilder. Therefore, it was not safe to assume this was a source-of-truth. This commit unexports the map of partial index predicates in TableMeta. Access to partial index predicates must now be done via `TableMeta.PartialIndexPredicate`. This function checks the catalog to determine if an index is a partial index, and panics if there is not a corresponding predicate expression in the partial index predicate map. This makes the function an actual a source-of-truth. Release note: None --- pkg/sql/opt/memo/expr.go | 2 +- pkg/sql/opt/memo/expr_format.go | 9 ++++--- pkg/sql/opt/metadata.go | 10 +++---- pkg/sql/opt/metadata_test.go | 7 ++--- pkg/sql/opt/norm/reject_nulls_funcs.go | 8 +++--- pkg/sql/opt/optbuilder/partial_index.go | 2 +- pkg/sql/opt/table_meta.go | 36 +++++++++++++++++++++---- pkg/sql/opt/xform/scan_index_iter.go | 2 +- pkg/sql/opt/xform/select_funcs.go | 2 +- 9 files changed, 54 insertions(+), 24 deletions(-) diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index d91af9257fae..5439b4fdbe39 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -658,7 +658,7 @@ func (s *ScanPrivate) IsLocking() bool { // index, nil is returned. func (s *ScanPrivate) PartialIndexPredicate(md *opt.Metadata) FiltersExpr { tabMeta := md.TableMeta(s.Table) - p, ok := tabMeta.PartialIndexPredicates[s.Index] + p, ok := tabMeta.PartialIndexPredicate(s.Index) if !ok { // The index is not a partial index. return nil diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 38c8ac33ea1d..abcc749f2343 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -352,17 +352,18 @@ func (f *ExprFmtCtx) formatRelational(e RelExpr, tp treeprinter.Node) { f.formatExpr(tab.ComputedCols[col], c.Child(f.ColumnString(col))) } } - if tab.PartialIndexPredicates != nil { + partialIndexPredicates := tab.PartialIndexPredicatesForFormattingOnly() + if partialIndexPredicates != nil { c := tp.Child("partial index predicates") - indexOrds := make([]cat.IndexOrdinal, 0, len(tab.PartialIndexPredicates)) - for ord := range tab.PartialIndexPredicates { + indexOrds := make([]cat.IndexOrdinal, 0, len(partialIndexPredicates)) + for ord := range partialIndexPredicates { indexOrds = append(indexOrds, ord) } sort.Ints(indexOrds) for _, ord := range indexOrds { name := string(tab.Table.Index(ord).Name()) f.Buffer.Reset() - f.formatScalarWithLabel(name, tab.PartialIndexPredicates[ord], c) + f.formatScalarWithLabel(name, partialIndexPredicates[ord], c) } } } diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 6a4ef3af62bd..51c644c37053 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -396,7 +396,7 @@ func (md *Metadata) AddTable(tab cat.Table, alias *tree.TableName) TableID { // ScalarExpr to new column IDs. It takes as arguments a ScalarExpr and a // mapping of old column IDs to new column IDs, and returns a new ScalarExpr. // This function is used when duplicating Constraints, ComputedCols, and -// PartialIndexPredicates. DuplicateTable requires this callback function, +// partialIndexPredicates. DuplicateTable requires this callback function, // rather than performing the remapping itself, because remapping column IDs // requires constructing new expressions with norm.Factory. The norm package // depends on opt, and cannot be imported here. @@ -450,9 +450,9 @@ func (md *Metadata) DuplicateTable( // Create new partial index predicate expressions by remapping the column // IDs in each ScalarExpr. var partialIndexPredicates map[cat.IndexOrdinal]ScalarExpr - if len(tabMeta.PartialIndexPredicates) > 0 { - partialIndexPredicates = make(map[cat.IndexOrdinal]ScalarExpr, len(tabMeta.PartialIndexPredicates)) - for idxOrd, e := range tabMeta.PartialIndexPredicates { + if len(tabMeta.partialIndexPredicates) > 0 { + partialIndexPredicates = make(map[cat.IndexOrdinal]ScalarExpr, len(tabMeta.partialIndexPredicates)) + for idxOrd, e := range tabMeta.partialIndexPredicates { partialIndexPredicates[idxOrd] = remapColumnIDs(e, colMap) } } @@ -464,7 +464,7 @@ func (md *Metadata) DuplicateTable( IgnoreForeignKeys: tabMeta.IgnoreForeignKeys, Constraints: constraints, ComputedCols: computedCols, - PartialIndexPredicates: partialIndexPredicates, + partialIndexPredicates: partialIndexPredicates, }) return newTabID diff --git a/pkg/sql/opt/metadata_test.go b/pkg/sql/opt/metadata_test.go index 6f98e25eff39..c1e4c3b45390 100644 --- a/pkg/sql/opt/metadata_test.go +++ b/pkg/sql/opt/metadata_test.go @@ -281,7 +281,7 @@ func TestIndexColumns(t *testing.T) { // TestDuplicateTable tests that we can extract a set of columns from an index ordinal. func TestDuplicateTable(t *testing.T) { cat := testcat.New() - _, err := cat.ExecuteDDL("CREATE TABLE a (b BOOL, b2 BOOL)") + _, err := cat.ExecuteDDL("CREATE TABLE a (b BOOL, b2 BOOL, INDEX (b2) WHERE b)") if err != nil { t.Fatal(err) } @@ -332,11 +332,12 @@ func TestDuplicateTable(t *testing.T) { t.Errorf("expected computed column to reference new column ID %d, got %d", dupB, col) } - if dupTabMeta.PartialIndexPredicates == nil || dupTabMeta.PartialIndexPredicates[1] == nil { + pred, isPartialIndex := dupTabMeta.PartialIndexPredicate(1) + if !isPartialIndex { t.Fatalf("expected partial index predicates to be duplicated") } - col = dupTabMeta.PartialIndexPredicates[1].(*memo.VariableExpr).Col + col = pred.(*memo.VariableExpr).Col if col == b { t.Errorf("expected partial index predicate to reference new column ID %d, got %d", dupB, col) } diff --git a/pkg/sql/opt/norm/reject_nulls_funcs.go b/pkg/sql/opt/norm/reject_nulls_funcs.go index 8ce83632baf1..b75eea8de5d9 100644 --- a/pkg/sql/opt/norm/reject_nulls_funcs.go +++ b/pkg/sql/opt/norm/reject_nulls_funcs.go @@ -331,9 +331,11 @@ func deriveScanRejectNullCols(in memo.RelExpr) opt.ColSet { scan := in.(*memo.ScanExpr) var rejectNullCols opt.ColSet - for _, pred := range md.TableMeta(scan.Table).PartialIndexPredicates { - predFilters := *pred.(*memo.FiltersExpr) - rejectNullCols.UnionWith(isNotNullCols(predFilters)) + for i, n := 0, md.Table(scan.Table).IndexCount(); i < n; i++ { + if pred, isPartialIndex := md.TableMeta(scan.Table).PartialIndexPredicate(i); isPartialIndex { + predFilters := *pred.(*memo.FiltersExpr) + rejectNullCols.UnionWith(isNotNullCols(predFilters)) + } } return rejectNullCols diff --git a/pkg/sql/opt/optbuilder/partial_index.go b/pkg/sql/opt/optbuilder/partial_index.go index 9e78103943fb..f2f99f9b5bac 100644 --- a/pkg/sql/opt/optbuilder/partial_index.go +++ b/pkg/sql/opt/optbuilder/partial_index.go @@ -25,7 +25,7 @@ import ( // Note: This function should only be used to build partial index or arbiter // predicate expressions that have only a table's ordinary columns in scope and // that are not part of the relational expression tree. For example, this is -// used to populate the TableMeta.PartialIndexPredicates cache and for +// used to populate the partial index predicates map in TableMeta and for // determining arbiter indexes in UPSERT and INSERT ON CONFLICT mutations. But // it is not used for building synthesized mutation columns that determine // whether to issue PUT or DEL operations on a partial index for a mutated row; diff --git a/pkg/sql/opt/table_meta.go b/pkg/sql/opt/table_meta.go index 35ada6385979..c9c67b92f768 100644 --- a/pkg/sql/opt/table_meta.go +++ b/pkg/sql/opt/table_meta.go @@ -156,11 +156,11 @@ type TableMeta struct { // Computed columns with non-immutable operators are omitted. ComputedCols map[ColumnID]ScalarExpr - // PartialIndexPredicates is a map from index ordinals on the table to + // partialIndexPredicates is a map from index ordinals on the table to // *FiltersExprs representing the predicate on the corresponding partial // index. If an index is not a partial index, it will not have an entry in // the map. - PartialIndexPredicates map[cat.IndexOrdinal]ScalarExpr + partialIndexPredicates map[cat.IndexOrdinal]ScalarExpr // anns annotates the table metadata with arbitrary data. anns [maxTableAnnIDCount]interface{} @@ -254,10 +254,36 @@ func (tm *TableMeta) AddComputedCol(colID ColumnID, computedCol ScalarExpr) { // AddPartialIndexPredicate adds a partial index predicate to the table's // metadata. func (tm *TableMeta) AddPartialIndexPredicate(ord cat.IndexOrdinal, pred ScalarExpr) { - if tm.PartialIndexPredicates == nil { - tm.PartialIndexPredicates = make(map[cat.IndexOrdinal]ScalarExpr) + if tm.partialIndexPredicates == nil { + tm.partialIndexPredicates = make(map[cat.IndexOrdinal]ScalarExpr) } - tm.PartialIndexPredicates[ord] = pred + tm.partialIndexPredicates[ord] = pred +} + +// PartialIndexPredicate returns the given index's predicate scalar expression, +// if the index is a partial index. Returns ok=false if the index is not a +// partial index. Panics if the index is a partial index but a predicate scalar +// expression does not exist in the table metadata. +func (tm *TableMeta) PartialIndexPredicate(ord cat.IndexOrdinal) (pred ScalarExpr, ok bool) { + if _, isPartialIndex := tm.Table.Index(ord).Predicate(); !isPartialIndex { + return nil, false + } + pred, ok = tm.partialIndexPredicates[ord] + if !ok { + panic(errors.AssertionFailedf("partial index predicate does not exist in table metadata")) + } + return pred, true +} + +// PartialIndexPredicatesForFormattingOnly returns the partialIndexPredicates +// map. +// +// WARNING: The returned map is NOT a source-of-truth for determining if an +// index is a partial index. This function should only be used to show the +// partial index expressions that have been built for a table when formatting +// opt expressions. Use PartialIndexPredicate in all other cases. +func (tm *TableMeta) PartialIndexPredicatesForFormattingOnly() map[cat.IndexOrdinal]ScalarExpr { + return tm.partialIndexPredicates } // TableAnnotation returns the given annotation that is associated with the diff --git a/pkg/sql/opt/xform/scan_index_iter.go b/pkg/sql/opt/xform/scan_index_iter.go index 99aef8acde30..c694369c1aee 100644 --- a/pkg/sql/opt/xform/scan_index_iter.go +++ b/pkg/sql/opt/xform/scan_index_iter.go @@ -213,7 +213,7 @@ func (it *scanIndexIter) ForEachStartingAfter(ord int, f enumerateIndexFunc) { continue } - pred, isPartialIndex := it.tabMeta.PartialIndexPredicates[ord] + pred, isPartialIndex := it.tabMeta.PartialIndexPredicate(ord) // Skip over partial indexes if rejectPartialIndexes is set. if it.hasRejectFlags(rejectPartialIndexes) && isPartialIndex { diff --git a/pkg/sql/opt/xform/select_funcs.go b/pkg/sql/opt/xform/select_funcs.go index 3a7eea0e7495..69c196d9ef9d 100644 --- a/pkg/sql/opt/xform/select_funcs.go +++ b/pkg/sql/opt/xform/select_funcs.go @@ -1655,7 +1655,7 @@ func (c *CustomFuncs) canMaybeConstrainIndexWithCols( // possible to generate an unconstrained partial index scan, which may // lead to better query plans. if _, isPartialIndex := index.Predicate(); isPartialIndex { - p, ok := tabMeta.PartialIndexPredicates[i] + p, ok := tabMeta.PartialIndexPredicate(i) if !ok { // A partial index predicate expression was not built for the // partial index. See Builder.buildScan for details on when this From 62ab822bc623c71f4c756ac80c18634e96f2e65d Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 29 Dec 2020 16:58:23 -0800 Subject: [PATCH 04/10] opt: move addPartialIndexPredicatesForTable to optbuilder/partial_index.go Release note: None --- pkg/sql/opt/optbuilder/partial_index.go | 75 +++++++++++++++++++++++++ pkg/sql/opt/optbuilder/select.go | 74 ------------------------ 2 files changed, 75 insertions(+), 74 deletions(-) diff --git a/pkg/sql/opt/optbuilder/partial_index.go b/pkg/sql/opt/optbuilder/partial_index.go index f2f99f9b5bac..413bae2e302f 100644 --- a/pkg/sql/opt/optbuilder/partial_index.go +++ b/pkg/sql/opt/optbuilder/partial_index.go @@ -13,11 +13,86 @@ package optbuilder import ( "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) +// addPartialIndexPredicatesForTable finds all partial indexes in the table and +// adds their predicates to the table metadata (see +// TableMeta.partialIndexPredicates). The predicates are converted from strings +// to ScalarExprs here. +// +// The predicates are used as "known truths" about table data. Any predicates +// containing non-immutable operators are omitted. +// +// scan is an optional argument that is a Scan expression on the table. If scan +// outputs all the ordinary columns in the table, we avoid constructing a new +// scan. A scan and its logical properties are required in order to fully +// normalize the partial index predicates. +func (b *Builder) addPartialIndexPredicatesForTable(tabMeta *opt.TableMeta, scan memo.RelExpr) { + tab := tabMeta.Table + + // Find the first partial index. + numIndexes := tab.IndexCount() + indexOrd := 0 + for ; indexOrd < numIndexes; indexOrd++ { + if _, ok := tab.Index(indexOrd).Predicate(); ok { + break + } + } + + // Return early if there are no partial indexes. Only partial indexes have + // predicates. + if indexOrd == numIndexes { + return + } + + // Construct a scan as the tableScope expr so that logical properties of the + // scan can be used to fully normalize the index predicate. + tableScope := b.allocScope() + tableScope.appendOrdinaryColumnsFromTable(tabMeta, &tabMeta.Alias) + + // If the optional scan argument was provided and it outputs all of the + // ordinary table columns, we use it as tableScope.expr. Otherwise, we must + // construct a new scan. Attaching a scan to tableScope.expr is required to + // fully normalize the partial index predicates with logical properties of + // the scan. + if scan != nil && tableScope.colSet().SubsetOf(scan.Relational().OutputCols) { + tableScope.expr = scan + } else { + tableScope.expr = b.factory.ConstructScan(&memo.ScanPrivate{ + Table: tabMeta.MetaID, + Cols: tableScope.colSet(), + }) + } + + // Skip to the first partial index we found above. + for ; indexOrd < numIndexes; indexOrd++ { + index := tab.Index(indexOrd) + pred, ok := index.Predicate() + + // If the index is not a partial index, do nothing. + if !ok { + continue + } + + expr, err := parser.ParseExpr(pred) + if err != nil { + panic(err) + } + + // Build the partial index predicate as a memo.FiltersExpr and add it + // to the table metadata. + predExpr, err := b.buildPartialIndexPredicate(tableScope, expr, "index predicate") + if err != nil { + panic(err) + } + tabMeta.AddPartialIndexPredicate(indexOrd, &predExpr) + } +} + // buildPartialIndexPredicate builds a memo.FiltersExpr from the given // tree.Expr. The expression must be of type Bool and it must be immutable. // Returns an error if any non-immutable operators are found. diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 6604913ce782..d155bfe68c42 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -688,80 +688,6 @@ func (b *Builder) addComputedColsForTable(tabMeta *opt.TableMeta) { } } -// addPartialIndexPredicatesForTable finds all partial indexes in the table and -// adds their predicates to the table metadata (see -// TableMeta.PartialIndexPredicates). The predicates are converted from strings -// to ScalarExprs here. -// -// The predicates are used as "known truths" about table data. Any predicates -// containing non-immutable operators are omitted. -// -// scan is an optional argument that is a Scan expression on the table. If scan -// outputs all the ordinary columns in the table, we avoid constructing a new -// scan. A scan and its logical properties are required in order to fully -// normalize the partial index predicates. -func (b *Builder) addPartialIndexPredicatesForTable(tabMeta *opt.TableMeta, scan memo.RelExpr) { - tab := tabMeta.Table - - // Find the first partial index. - numIndexes := tab.IndexCount() - indexOrd := 0 - for ; indexOrd < numIndexes; indexOrd++ { - if _, ok := tab.Index(indexOrd).Predicate(); ok { - break - } - } - - // Return early if there are no partial indexes. Only partial indexes have - // predicates. - if indexOrd == numIndexes { - return - } - - // Construct a scan as the tableScope expr so that logical properties of the - // scan can be used to fully normalize the index predicate. - tableScope := b.allocScope() - tableScope.appendOrdinaryColumnsFromTable(tabMeta, &tabMeta.Alias) - - // If the optional scan argument was provided and it outputs all of the - // ordinary table columns, we use it as tableScope.expr. Otherwise, we must - // construct a new scan. Attaching a scan to tableScope.expr is required to - // fully normalize the partial index predicates with logical properties of - // the scan. - if scan != nil && tableScope.colSet().SubsetOf(scan.Relational().OutputCols) { - tableScope.expr = scan - } else { - tableScope.expr = b.factory.ConstructScan(&memo.ScanPrivate{ - Table: tabMeta.MetaID, - Cols: tableScope.colSet(), - }) - } - - // Skip to the first partial index we found above. - for ; indexOrd < numIndexes; indexOrd++ { - index := tab.Index(indexOrd) - pred, ok := index.Predicate() - - // If the index is not a partial index, do nothing. - if !ok { - continue - } - - expr, err := parser.ParseExpr(pred) - if err != nil { - panic(err) - } - - // Build the partial index predicate as a memo.FiltersExpr and add it - // to the table metadata. - predExpr, err := b.buildPartialIndexPredicate(tableScope, expr, "index predicate") - if err != nil { - panic(err) - } - tabMeta.AddPartialIndexPredicate(indexOrd, &predExpr) - } -} - func (b *Builder) buildSequenceSelect( seq cat.Sequence, seqName *tree.TableName, inScope *scope, ) (outScope *scope) { From 4e140b62fbe766ea1bb735700e9e564686baa67b Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 29 Dec 2020 12:02:45 -0800 Subject: [PATCH 05/10] opt: prune update/upsert fetch columns not needed for partial indexes Indexed columns of partial indexes are now only fetched for UPDATE and UPSERT operations when needed. They are pruned in cases where it is guaranteed that they are not needed to build old or new index entries. For example, consider the table and UPDATE: CREATE TABLE t ( a INT PRIMARY KEY, b INT, c INT, d INT, INDEX (b) WHERE c > 0, FAMILY (a), FAMILY (b), FAMILY (c), FAMILY (d) ) UPDATE t SET d = d + 1 WHERE a = 1 The partial index is guaranteed not to change with this UPDATE because neither its indexed columns nor the columns referenced in its predicate are mutating. Therefore, the existing values of b do not need to be fetched to maintain the state of the partial index. Furthermore, the primary index does require the existing values of b because no columns in b's family are mutating. So, b can be pruned from the UPDATE's fetch columns. Release note (performance improvement): Previously, indexed columns of partial indexes were always fetched for UPDATEs and UPSERTs. Now they are only fetched if they are required for maintaining the state of the index. If an UPDATE or UPSERT mutates columns that are neither indexed by a partial index nor referenced in a partial index predicate, they will no longer be fetched (assuming that they are not needed to maintain the state of other indexes, including the primary index). --- .../testdata/logic_test/partial_index | 48 +++ pkg/sql/opt/norm/prune_cols_funcs.go | 28 +- pkg/sql/opt/norm/testdata/rules/prune_cols | 393 +++++++++++++++++- pkg/sql/opt/optbuilder/insert.go | 5 + pkg/sql/opt/optbuilder/partial_index.go | 11 +- pkg/sql/opt/optbuilder/select.go | 7 +- pkg/sql/opt/optbuilder/update.go | 5 + 7 files changed, 459 insertions(+), 38 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/partial_index b/pkg/sql/logictest/testdata/logic_test/partial_index index c838197fdc9b..b3b5b152ec6a 100644 --- a/pkg/sql/logictest/testdata/logic_test/partial_index +++ b/pkg/sql/logictest/testdata/logic_test/partial_index @@ -1281,6 +1281,54 @@ SELECT * FROM inv_c@i WHERE j @> '{"x": "y"}' AND s IN ('foo', 'bar') ORDER BY k 1 {"num": 1, "x": "y"} foo 3 {"num": 3, "x": "y"} bar +# Updates and Upserts with fetch columns pruned. + +statement ok +CREATE TABLE prune ( + a INT PRIMARY KEY, + b INT, + c INT, + d INT, + INDEX idx (b) WHERE c > 0, + FAMILY (a), + FAMILY (b), + FAMILY (c), + FAMILY (d) +) + +statement ok +INSERT INTO prune (a, b, c, d) VALUES (1, 2, 3, 4) + +# Test that an update is successful when fetch columns b and c are pruned +# because an update to idx is not required. +statement ok +UPDATE prune SET d = d + 1 WHERE a = 1 + +query IIII rowsort +SELECT * FROM prune@idx WHERE c > 0 +---- +1 2 3 5 + +# Test that an upsert is successful when fetch columns b and c are pruned +# because an update to idx is not required. +statement ok +UPSERT INTO prune (a, d) VALUES (1, 6) + +query IIII rowsort +SELECT * FROM prune@idx WHERE c > 0 +---- +1 2 3 6 + +# Test that an upsert is successful when fetch columns b and c are pruned +# because an update to idx is not required. +statement ok +INSERT INTO prune (a, d) VALUES (1, 6) ON CONFLICT (a) DO UPDATE SET d = 7 + +query IIII rowsort +SELECT * FROM prune@idx WHERE c > 0 +---- +1 2 3 7 + # Regression tests for #52318. Mutations on partial indexes in the # DELETE_AND_WRITE_ONLY state should update the indexes correctly. subtest regression_52318 diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index 831f982ba586..f65672f32d5d 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -144,26 +144,26 @@ func (c *CustomFuncs) NeededMutationFetchCols( // Make sure to consider indexes that are being added or dropped. for i, n := 0, tabMeta.Table.DeletableIndexCount(); i < n; i++ { - // If the columns being updated are not part of the index and the - // index is not a partial index, then the update does not require - // changes to the index. Partial indexes may be updated (even when a - // column in the index is not changing) when rows that were not - // previously in the index must be added to the index because they - // now satisfy the partial index predicate. + // If the columns being updated are not part of the index, then the + // update does not require changes to the index. Partial indexes may + // be updated (even when a column in the index is not changing) when + // the predicate references columns that are being updated. For + // example, rows that were not previously in the index must be added + // to the index because they now satisfy the partial index + // predicate, requiring the index columns to be fetched. // // Note that we use the set of index columns where the virtual // columns have been mapped to their source columns. Virtual columns // are never part of the updated columns. Updates to source columns // trigger index changes. - // - // TODO(mgartner): Index columns are not necessary when neither the - // index columns nor the columns referenced in the partial index - // predicate are being updated. We should prune mutation fetch - // columns when this is the case, rather than always marking index - // columns of partial indexes as "needed". indexCols := tabMeta.IndexColumnsMapVirtual(i) - _, isPartialIndex := tabMeta.Table.Index(i).Predicate() - if !indexCols.Intersects(updateCols) && !isPartialIndex { + pred, isPartialIndex := tabMeta.PartialIndexPredicate(i) + indexAndPredCols := indexCols.Copy() + if isPartialIndex { + predFilters := *pred.(*memo.FiltersExpr) + indexAndPredCols.UnionWith(predFilters.OuterCols()) + } + if !indexAndPredCols.Intersects(updateCols) { continue } diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index bde5476683f1..ef54a006d307 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -48,9 +48,11 @@ CREATE TABLE partial_indexes ( a INT PRIMARY KEY, b INT, c STRING, + d INT, FAMILY (a), FAMILY (b), FAMILY (c), + FAMILY (d), INDEX (c) WHERE b > 1 ) ---- @@ -2058,51 +2060,404 @@ update computed ├── c_new:13 + 1 [as=column14:14, outer=(13), immutable] └── c_new:13 + 10 [as=column15:15, outer=(13), immutable] +# Prune UPDATE fetch columns when the partial index indexes the column but +# neither the column nor the columns referenced in the partial index predicate +# are mutating. +norm expect=(PruneMutationFetchCols,PruneMutationInputCols) +UPDATE partial_indexes SET d = d + 1 WHERE a = 1 +---- +update partial_indexes + ├── columns: + ├── fetch columns: a:6 d:9 + ├── update-mapping: + │ └── d_new:11 => d:4 + ├── partial index put columns: partial_index_put1:12 + ├── partial index del columns: partial_index_put1:12 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:12 d_new:11 a:6!null d:9 + ├── cardinality: [0 - 1] + ├── immutable + ├── key: () + ├── fd: ()-->(6,9,11,12) + ├── select + │ ├── columns: a:6!null b:7 d:9 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(6,7,9) + │ ├── scan partial_indexes + │ │ ├── columns: a:6!null b:7 d:9 + │ │ ├── partial index predicates + │ │ │ └── secondary: filters + │ │ │ └── b:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7,9) + │ └── filters + │ └── a:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + └── projections + ├── b:7 > 1 [as=partial_index_put1:12, outer=(7)] + └── d:9 + 1 [as=d_new:11, outer=(9), immutable] + +# Do not prune the indexed column c when a column in the partial index +# predicate, b, is being updated. +norm expect-not=PruneMutationFetchCols +UPDATE partial_indexes SET d = d + 1, b = 2 WHERE a = 1 +---- +update partial_indexes + ├── columns: + ├── fetch columns: a:6 b:7 c:8 d:9 + ├── update-mapping: + │ ├── b_new:12 => b:2 + │ └── d_new:11 => d:4 + ├── partial index put columns: partial_index_put1:13 + ├── partial index del columns: partial_index_del1:14 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:13!null partial_index_del1:14 d_new:11 b_new:12!null a:6!null b:7 c:8 d:9 + ├── cardinality: [0 - 1] + ├── immutable + ├── key: () + ├── fd: ()-->(6-9,11-14) + ├── select + │ ├── columns: a:6!null b:7 c:8 d:9 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(6-9) + │ ├── scan partial_indexes + │ │ ├── columns: a:6!null b:7 c:8 d:9 + │ │ ├── partial index predicates + │ │ │ └── secondary: filters + │ │ │ └── b:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7-9) + │ └── filters + │ └── a:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + └── projections + ├── true [as=partial_index_put1:13] + ├── b:7 > 1 [as=partial_index_del1:14, outer=(7)] + ├── d:9 + 1 [as=d_new:11, outer=(9), immutable] + └── 2 [as=b_new:12] + +# Prune UPSERT fetch columns when the partial index indexes the column but +# neither the column nor the columns referenced in the partial index predicate +# are mutating. +norm expect=(PruneMutationFetchCols,PruneMutationInputCols) +UPSERT INTO partial_indexes (a, d) VALUES (1, 2) +---- +upsert partial_indexes + ├── columns: + ├── arbiter indexes: primary + ├── canary column: a:10 + ├── fetch columns: a:10 d:13 + ├── insert-mapping: + │ ├── column1:6 => a:1 + │ ├── column8:8 => b:2 + │ ├── column9:9 => c:3 + │ └── column2:7 => d:4 + ├── update-mapping: + │ └── column2:7 => d:4 + ├── partial index put columns: partial_index_put1:18 + ├── partial index del columns: partial_index_del1:19 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:18 partial_index_del1:19 column1:6!null column2:7!null column8:8 column9:9 a:10 d:13 + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(6-10,13,18,19) + ├── left-join (cross) + │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 d:13 + │ ├── cardinality: [1 - 1] + │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + │ ├── key: () + │ ├── fd: ()-->(6-11,13) + │ ├── values + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(6-9) + │ │ └── (1, 2, NULL, NULL) + │ ├── select + │ │ ├── columns: a:10!null b:11 d:13 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(10,11,13) + │ │ ├── scan partial_indexes + │ │ │ ├── columns: a:10!null b:11 d:13 + │ │ │ ├── partial index predicates + │ │ │ │ └── secondary: filters + │ │ │ │ └── b:11 > 1 [outer=(11), constraints=(/11: [/2 - ]; tight)] + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11,13) + │ │ └── filters + │ │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] + │ └── filters (true) + └── projections + ├── CASE WHEN a:10 IS NULL THEN column8:8 ELSE b:11 END > 1 [as=partial_index_put1:18, outer=(8,10,11)] + └── b:11 > 1 [as=partial_index_del1:19, outer=(11)] + +# Do not prune the indexed column c when a column in the partial index +# predicate, b, is being updated. +norm expect-not=PruneMutationFetchCols +UPSERT INTO partial_indexes (a, b, d) VALUES (1, 2, 3) +---- +upsert partial_indexes + ├── columns: + ├── arbiter indexes: primary + ├── canary column: a:10 + ├── fetch columns: a:10 b:11 c:12 d:13 + ├── insert-mapping: + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column9:9 => c:3 + │ └── column3:8 => d:4 + ├── update-mapping: + │ ├── column2:7 => b:2 + │ └── column3:8 => d:4 + ├── partial index put columns: partial_index_put1:17 + ├── partial index del columns: partial_index_del1:18 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:17!null partial_index_del1:18 column1:6!null column2:7!null column3:8!null column9:9 a:10 b:11 c:12 d:13 + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(6-13,17,18) + ├── left-join (cross) + │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 a:10 b:11 c:12 d:13 + │ ├── cardinality: [1 - 1] + │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + │ ├── key: () + │ ├── fd: ()-->(6-13) + │ ├── values + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(6-9) + │ │ └── (1, 2, 3, NULL) + │ ├── select + │ │ ├── columns: a:10!null b:11 c:12 d:13 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(10-13) + │ │ ├── scan partial_indexes + │ │ │ ├── columns: a:10!null b:11 c:12 d:13 + │ │ │ ├── partial index predicates + │ │ │ │ └── secondary: filters + │ │ │ │ └── b:11 > 1 [outer=(11), constraints=(/11: [/2 - ]; tight)] + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11-13) + │ │ └── filters + │ │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] + │ └── filters (true) + └── projections + ├── column2:7 > 1 [as=partial_index_put1:17, outer=(7)] + └── b:11 > 1 [as=partial_index_del1:18, outer=(11)] + +# Prune INSERT ON CONFLICT DO UPDATE fetch columns when the partial index +# indexes the column but neither the column nor the columns referenced in the +# partial index predicate are mutating. +norm expect=(PruneMutationFetchCols,PruneMutationInputCols) +INSERT INTO partial_indexes (a, d) VALUES (1, 2) ON CONFLICT (a) DO UPDATE SET d = 3 +---- +upsert partial_indexes + ├── columns: + ├── arbiter indexes: primary + ├── canary column: a:10 + ├── fetch columns: a:10 d:13 + ├── insert-mapping: + │ ├── column1:6 => a:1 + │ ├── column8:8 => b:2 + │ ├── column9:9 => c:3 + │ └── column2:7 => d:4 + ├── update-mapping: + │ └── upsert_d:19 => d:4 + ├── partial index put columns: partial_index_put1:20 + ├── partial index del columns: partial_index_del1:21 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:20 partial_index_del1:21 upsert_d:19!null column1:6!null column2:7!null column8:8 column9:9 a:10 d:13 + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(6-10,13,19-21) + ├── left-join (cross) + │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 d:13 + │ ├── cardinality: [1 - 1] + │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + │ ├── key: () + │ ├── fd: ()-->(6-11,13) + │ ├── values + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(6-9) + │ │ └── (1, 2, NULL, NULL) + │ ├── select + │ │ ├── columns: a:10!null b:11 d:13 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(10,11,13) + │ │ ├── scan partial_indexes + │ │ │ ├── columns: a:10!null b:11 d:13 + │ │ │ ├── partial index predicates + │ │ │ │ └── secondary: filters + │ │ │ │ └── b:11 > 1 [outer=(11), constraints=(/11: [/2 - ]; tight)] + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11,13) + │ │ └── filters + │ │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] + │ └── filters (true) + └── projections + ├── CASE WHEN a:10 IS NULL THEN column8:8 ELSE b:11 END > 1 [as=partial_index_put1:20, outer=(8,10,11)] + ├── b:11 > 1 [as=partial_index_del1:21, outer=(11)] + └── CASE WHEN a:10 IS NULL THEN column2:7 ELSE 3 END [as=upsert_d:19, outer=(7,10)] + +# Do not prune the indexed column c when a column in the partial index +# predicate, b, is being updated. +norm expect-not=PruneMutationFetchCols +INSERT INTO partial_indexes (a, d) VALUES (1, 2) ON CONFLICT (a) DO UPDATE SET b = 3, d = 4 +---- +upsert partial_indexes + ├── columns: + ├── arbiter indexes: primary + ├── canary column: a:10 + ├── fetch columns: a:10 b:11 c:12 d:13 + ├── insert-mapping: + │ ├── column1:6 => a:1 + │ ├── column8:8 => b:2 + │ ├── column9:9 => c:3 + │ └── column2:7 => d:4 + ├── update-mapping: + │ ├── upsert_b:18 => b:2 + │ └── upsert_d:20 => d:4 + ├── partial index put columns: partial_index_put1:21 + ├── partial index del columns: partial_index_del1:22 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:21 partial_index_del1:22 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 upsert_b:18 upsert_d:20!null + ├── cardinality: [1 - 1] + ├── key: () + ├── fd: ()-->(6-13,18,20-22) + ├── project + │ ├── columns: upsert_b:18 upsert_d:20!null column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(6-13,18,20) + │ ├── left-join (cross) + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 + │ │ ├── cardinality: [1 - 1] + │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) + │ │ ├── key: () + │ │ ├── fd: ()-->(6-13) + │ │ ├── values + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── key: () + │ │ │ ├── fd: ()-->(6-9) + │ │ │ └── (1, 2, NULL, NULL) + │ │ ├── select + │ │ │ ├── columns: a:10!null b:11 c:12 d:13 + │ │ │ ├── cardinality: [0 - 1] + │ │ │ ├── key: () + │ │ │ ├── fd: ()-->(10-13) + │ │ │ ├── scan partial_indexes + │ │ │ │ ├── columns: a:10!null b:11 c:12 d:13 + │ │ │ │ ├── partial index predicates + │ │ │ │ │ └── secondary: filters + │ │ │ │ │ └── b:11 > 1 [outer=(11), constraints=(/11: [/2 - ]; tight)] + │ │ │ │ ├── key: (10) + │ │ │ │ └── fd: (10)-->(11-13) + │ │ │ └── filters + │ │ │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] + │ │ └── filters (true) + │ └── projections + │ ├── CASE WHEN a:10 IS NULL THEN column8:8 ELSE 3 END [as=upsert_b:18, outer=(8,10)] + │ └── CASE WHEN a:10 IS NULL THEN column2:7 ELSE 4 END [as=upsert_d:20, outer=(7,10)] + └── projections + ├── upsert_b:18 > 1 [as=partial_index_put1:21, outer=(18)] + └── b:11 > 1 [as=partial_index_del1:22, outer=(11)] + +# Do not prune DELETE fetch columns. +norm +DELETE FROM partial_indexes WHERE a = 1 +---- +delete partial_indexes + ├── columns: + ├── fetch columns: a:6 c:8 + ├── partial index del columns: partial_index_del1:11 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_del1:11 a:6!null c:8 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(6,8,11) + ├── select + │ ├── columns: a:6!null b:7 c:8 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(6-8) + │ ├── scan partial_indexes + │ │ ├── columns: a:6!null b:7 c:8 + │ │ ├── partial index predicates + │ │ │ └── secondary: filters + │ │ │ └── b:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7,8) + │ └── filters + │ └── a:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + └── projections + └── b:7 > 1 [as=partial_index_del1:11, outer=(7)] + # Do not prune columns that are required for evaluating partial index # predicates. -norm expect-not=PruneMutationFetchCols +norm UPDATE partial_indexes SET b = b + 1 WHERE a = 1 ---- update partial_indexes ├── columns: - ├── fetch columns: a:5 b:6 c:7 + ├── fetch columns: a:6 b:7 c:8 ├── update-mapping: - │ └── b_new:9 => b:2 - ├── partial index put columns: partial_index_put1:10 - ├── partial index del columns: partial_index_del1:11 + │ └── b_new:11 => b:2 + ├── partial index put columns: partial_index_put1:12 + ├── partial index del columns: partial_index_del1:13 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: partial_index_put1:10 partial_index_del1:11 a:5!null b:6 c:7 b_new:9 + ├── columns: partial_index_put1:12 partial_index_del1:13 a:6!null b:7 c:8 b_new:11 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5-7,9-11) + ├── fd: ()-->(6-8,11-13) ├── project - │ ├── columns: b_new:9 a:5!null b:6 c:7 + │ ├── columns: b_new:11 a:6!null b:7 c:8 │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(5-7,9) + │ ├── fd: ()-->(6-8,11) │ ├── select - │ │ ├── columns: a:5!null b:6 c:7 + │ │ ├── columns: a:6!null b:7 c:8 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5-7) + │ │ ├── fd: ()-->(6-8) │ │ ├── scan partial_indexes - │ │ │ ├── columns: a:5!null b:6 c:7 + │ │ │ ├── columns: a:6!null b:7 c:8 │ │ │ ├── partial index predicates │ │ │ │ └── secondary: filters - │ │ │ │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6,7) + │ │ │ │ └── b:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7,8) │ │ └── filters - │ │ └── a:5 = 1 [outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] + │ │ └── a:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] │ └── projections - │ └── b:6 + 1 [as=b_new:9, outer=(6), immutable] + │ └── b:7 + 1 [as=b_new:11, outer=(7), immutable] └── projections - ├── b_new:9 > 1 [as=partial_index_put1:10, outer=(9)] - └── b:6 > 1 [as=partial_index_del1:11, outer=(6)] + ├── b_new:11 > 1 [as=partial_index_put1:12, outer=(11)] + └── b:7 > 1 [as=partial_index_del1:13, outer=(7)] # Prune secondary family column not needed for the update. norm expect=(PruneMutationFetchCols,PruneMutationInputCols) diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index aa929e258714..cdf095caad30 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -1061,6 +1061,11 @@ func (mb *mutationBuilder) buildUpsert(returning tree.ReturningExprs) { // Add any check constraint boolean columns to the input. mb.addCheckConstraintCols() + // Add the partial index predicate expressions to the table metadata. + // These expressions are used to prune fetch columns during + // normalization. + mb.b.addPartialIndexPredicatesForTable(mb.md.TableMeta(mb.tabID), nil /* scan */, true /* includeDeletable */) + // Project partial index PUT and DEL boolean columns. // // In some cases existing rows may not be fetched for an UPSERT (see diff --git a/pkg/sql/opt/optbuilder/partial_index.go b/pkg/sql/opt/optbuilder/partial_index.go index 413bae2e302f..c60cc64136c7 100644 --- a/pkg/sql/opt/optbuilder/partial_index.go +++ b/pkg/sql/opt/optbuilder/partial_index.go @@ -31,11 +31,18 @@ import ( // outputs all the ordinary columns in the table, we avoid constructing a new // scan. A scan and its logical properties are required in order to fully // normalize the partial index predicates. -func (b *Builder) addPartialIndexPredicatesForTable(tabMeta *opt.TableMeta, scan memo.RelExpr) { +func (b *Builder) addPartialIndexPredicatesForTable( + tabMeta *opt.TableMeta, scan memo.RelExpr, includeDeletable bool, +) { tab := tabMeta.Table + var numIndexes int + if includeDeletable { + numIndexes = tab.DeletableIndexCount() + } else { + numIndexes = tab.IndexCount() + } // Find the first partial index. - numIndexes := tab.IndexCount() indexOrd := 0 for ; indexOrd < numIndexes; indexOrd++ { if _, ok := tab.Index(indexOrd).Predicate(); ok { diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index d155bfe68c42..3a74a991df95 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -536,9 +536,10 @@ func (b *Builder) buildScan( outScope.expr = b.factory.ConstructScan(&private) // Add the partial indexes after constructing the scan so we can use the - // logical properties of the scan to fully normalize the index - // predicates. - b.addPartialIndexPredicatesForTable(tabMeta, outScope.expr) + // logical properties of the scan to fully normalize the index predicates. + // We don't need to add deletable partial index predicates in the context of + // a scan. + b.addPartialIndexPredicatesForTable(tabMeta, outScope.expr, false /* includeDeletable */) if !virtualColIDs.Empty() { // Project the expressions for the virtual columns (and pass through all diff --git a/pkg/sql/opt/optbuilder/update.go b/pkg/sql/opt/optbuilder/update.go index e91158ef185a..a38f3370a795 100644 --- a/pkg/sql/opt/optbuilder/update.go +++ b/pkg/sql/opt/optbuilder/update.go @@ -336,6 +336,11 @@ func (mb *mutationBuilder) buildUpdate(returning tree.ReturningExprs) { mb.addCheckConstraintCols() + // Add the partial index predicate expressions to the table metadata. + // These expressions are used to prune fetch columns during + // normalization. + mb.b.addPartialIndexPredicatesForTable(mb.md.TableMeta(mb.tabID), nil /* scan */, true /* includeDeletable */) + // Project partial index PUT and DEL boolean columns. mb.projectPartialIndexPutAndDelCols(preCheckScope, mb.fetchScope) From 039fb1b669ba25f10733eba8dc3ec28fca7a1a24 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 29 Dec 2020 16:30:58 -0800 Subject: [PATCH 06/10] opt: normalize partial index PUT/DEL projections to false The `SimplifyPartialIndexProjections` normalization rule has been added that normalizes synthesized partial index PUT and DEL columns to False when it is guaranteed that a mutation will not require changes to the associated partial index. This normalization can lead to further normalizations, such as pruning columns that the synthesized projections relied on. The motivation for this change is to allow fully disjoint updates to different columns in the same row, when the columns are split across different families. By pruning columns not needed to maintain a partial index, we're not forced to scan all column families. This can ultimately reduce contention during updates. Release note (performance improvement): UPDATE operations on tables with partial indexes no longer evaluate partial index predicate expressions when it is guaranteed that the operation will not alter the state of the partial index. In some cases, this can eliminate fetching the existing value of columns that are referenced in partial index predicates. --- pkg/sql/opt/norm/BUILD.bazel | 1 + pkg/sql/opt/norm/mutation_funcs.go | 119 ++++++++++ pkg/sql/opt/norm/prune_cols_funcs.go | 15 +- pkg/sql/opt/norm/rules/mutation.opt | 38 +++ pkg/sql/opt/norm/testdata/rules/mutation | 258 +++++++++++++++++++++ pkg/sql/opt/norm/testdata/rules/prune_cols | 12 +- 6 files changed, 435 insertions(+), 8 deletions(-) create mode 100644 pkg/sql/opt/norm/mutation_funcs.go create mode 100644 pkg/sql/opt/norm/testdata/rules/mutation diff --git a/pkg/sql/opt/norm/BUILD.bazel b/pkg/sql/opt/norm/BUILD.bazel index 28af1cffb2d6..1355bdbd12fb 100644 --- a/pkg/sql/opt/norm/BUILD.bazel +++ b/pkg/sql/opt/norm/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "join_funcs.go", "limit_funcs.go", "list_sorter.go", + "mutation_funcs.go", "ordering_funcs.go", "project_builder.go", "project_funcs.go", diff --git a/pkg/sql/opt/norm/mutation_funcs.go b/pkg/sql/opt/norm/mutation_funcs.go new file mode 100644 index 000000000000..c1800fc60f33 --- /dev/null +++ b/pkg/sql/opt/norm/mutation_funcs.go @@ -0,0 +1,119 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package norm + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" +) + +// SimplifiablePartialIndexProjectCols returns the set of projected partial +// index PUT and DEL columns with expressions that can be simplified to false. +// These projected expressions can only be simplified to false when an UPDATE +// mutates neither the associated index's columns nor the columns referenced in +// the partial index predicate. +func (c *CustomFuncs) SimplifiablePartialIndexProjectCols( + private *memo.MutationPrivate, + uniqueChecks memo.UniqueChecksExpr, + fkChecks memo.FKChecksExpr, + projections memo.ProjectionsExpr, +) opt.ColSet { + tabMeta := c.mem.Metadata().TableMeta(private.Table) + + // Determine the set of target table columns that need to be updated. Notice + // that we collect the target table column IDs, not the update column IDs. + var updateCols opt.ColSet + for ord, col := range private.UpdateCols { + if col != 0 { + updateCols.Add(tabMeta.MetaID.ColumnID(ord)) + } + } + + // Determine the set of columns needed for the mutation operator, excluding + // the partial index PUT and DEL columns. + neededMutationCols := c.neededMutationCols(private, uniqueChecks, fkChecks, false /* includePartialIndexCols */) + + // Determine the set of project columns that are already simplified to + // false. + var simplifiedProjectCols opt.ColSet + for i := range projections { + project := &projections[i] + if project.Element == memo.FalseSingleton { + simplifiedProjectCols.Add(project.Col) + } + } + + // Columns that are required by the mutation operator and columns that + // have already been simplified to false are ineligible to be simplified. + ineligibleCols := neededMutationCols.Union(simplifiedProjectCols) + + // ord is an ordinal into the mutation's PartialIndexPutCols and + // PartialIndexDelCols, which both have entries for each partial index + // defined on the table. + ord := -1 + var cols opt.ColSet + for i, n := 0, tabMeta.Table.DeletableIndexCount(); i < n; i++ { + pred, isPartialIndex := tabMeta.PartialIndexPredicate(i) + + // Skip non-partial indexes. + if !isPartialIndex { + continue + } + ord++ + + // If the columns being updated are part of the index or referenced in + // the partial index predicate, then updates to the index may be + // required. Therefore, the partial index PUT and DEL columns cannot be + // simplified. + // + // Note that we use the set of index columns where the virtual + // columns have been mapped to their source columns. Virtual columns + // are never part of the updated columns. Updates to source columns + // trigger index changes. + predFilters := *pred.(*memo.FiltersExpr) + indexAndPredCols := tabMeta.IndexColumnsMapVirtual(i) + indexAndPredCols.UnionWith(predFilters.OuterCols()) + if indexAndPredCols.Intersects(updateCols) { + continue + } + + // Add the projected PUT column if it is eligible to be simplified. + putCol := private.PartialIndexPutCols[ord] + if !ineligibleCols.Contains(putCol) { + cols.Add(putCol) + } + + // Add the projected DEL column if it is eligible to be simplified. + delCol := private.PartialIndexDelCols[ord] + if !ineligibleCols.Contains(delCol) { + cols.Add(delCol) + } + } + + return cols +} + +// SimplifyPartialIndexProjections returns a new projection expression with any +// projected column's expression simplified to false if the column exists in +// simplifiableCols. +func (c *CustomFuncs) SimplifyPartialIndexProjections( + projections memo.ProjectionsExpr, simplifiableCols opt.ColSet, +) memo.ProjectionsExpr { + simplified := make(memo.ProjectionsExpr, len(projections)) + for i := range projections { + if col := projections[i].Col; simplifiableCols.Contains(col) { + simplified[i] = c.f.ConstructProjectionsItem(memo.FalseSingleton, col) + } else { + simplified[i] = projections[i] + } + } + return simplified +} diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index f65672f32d5d..c7aa2303aacf 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -44,6 +44,15 @@ func (c *CustomFuncs) NeededExplainCols(private *memo.ExplainPrivate) opt.ColSet // in turn trigger the PruneMutationInputCols rule. func (c *CustomFuncs) NeededMutationCols( private *memo.MutationPrivate, uniqueChecks memo.UniqueChecksExpr, fkChecks memo.FKChecksExpr, +) opt.ColSet { + return c.neededMutationCols(private, uniqueChecks, fkChecks, true /* includePartialIndexCols */) +} + +func (c *CustomFuncs) neededMutationCols( + private *memo.MutationPrivate, + uniqueChecks memo.UniqueChecksExpr, + fkChecks memo.FKChecksExpr, + includePartialIndexCols bool, ) opt.ColSet { var cols opt.ColSet @@ -60,8 +69,10 @@ func (c *CustomFuncs) NeededMutationCols( addCols(private.FetchCols) addCols(private.UpdateCols) addCols(private.CheckCols) - addCols(private.PartialIndexPutCols) - addCols(private.PartialIndexDelCols) + if includePartialIndexCols { + addCols(private.PartialIndexPutCols) + addCols(private.PartialIndexDelCols) + } addCols(private.ReturnCols) addCols(opt.OptionalColList(private.PassthroughCols)) if private.CanaryCol != 0 { diff --git a/pkg/sql/opt/norm/rules/mutation.opt b/pkg/sql/opt/norm/rules/mutation.opt index e69de29bb2d1..e66c33278474 100644 --- a/pkg/sql/opt/norm/rules/mutation.opt +++ b/pkg/sql/opt/norm/rules/mutation.opt @@ -0,0 +1,38 @@ +# ============================================================================= +# mutation.opt contains normalization rules for the mutation operators. +# ============================================================================= + +# SimplifyPartialIndexProjections converts partial index PUT and DEL projected +# expressions to false when it is guaranteed that the mutation will not require +# changes to the associated partial index. These projected expressions can only +# be simplified to false when an UPDATE mutates neither the associated index's +# columns nor the columns referenced in the partial index predicate. +[SimplifyPartialIndexProjections, Normalize] +(Update + $project:(Project $input:* $projections:* $passthrough:*) + $uniqueChecks:* + $fkChecks:* + $mutationPrivate:* & + ^(ColsAreEmpty + $simplifiableCols:(SimplifiablePartialIndexProjectCols + $mutationPrivate + $uniqueChecks + $fkChecks + $projections + ) + ) +) +=> +(Update + (Project + $input + (SimplifyPartialIndexProjections + $projections + $simplifiableCols + ) + $passthrough + ) + $uniqueChecks + $fkChecks + $mutationPrivate +) diff --git a/pkg/sql/opt/norm/testdata/rules/mutation b/pkg/sql/opt/norm/testdata/rules/mutation new file mode 100644 index 000000000000..5c4962779360 --- /dev/null +++ b/pkg/sql/opt/norm/testdata/rules/mutation @@ -0,0 +1,258 @@ +# -------------------------------------------------- +# SimplifyPartialIndexProjections +# -------------------------------------------------- + +exec-ddl +CREATE TABLE t ( + k INT PRIMARY KEY, + a INT, + b INT, + c INT, + d INT, + e INT, + f INT, + g INT, + h BOOL, + INDEX (a), + INDEX (c) WHERE d > 1, + INDEX (e) WHERE f > 1 AND g > 1, + INDEX (b), + INDEX (d) WHERE c > 1 +) +---- + +# Simplify UPDATE partial index put/del column to false when the indexed columns +# and columns referenced in predicates are not mutating. +norm expect=SimplifyPartialIndexProjections +UPDATE t SET a = 2, b = 2 WHERE k = 1 +---- +update t + ├── columns: + ├── fetch columns: k:11 a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── update-mapping: + │ ├── a_new:21 => a:2 + │ └── a_new:21 => b:3 + ├── partial index put columns: partial_index_put1:22 partial_index_put2:23 partial_index_put3:24 + ├── partial index del columns: partial_index_put1:22 partial_index_put2:23 partial_index_put3:24 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:22!null partial_index_put2:23!null partial_index_put3:24!null a_new:21!null k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(11-19,21-24) + ├── select + │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11-19) + │ ├── scan t + │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ ├── partial index predicates + │ │ │ ├── secondary: filters + │ │ │ │ └── d:15 > 1 [outer=(15), constraints=(/15: [/2 - ]; tight)] + │ │ │ ├── secondary: filters + │ │ │ │ ├── f:17 > 1 [outer=(17), constraints=(/17: [/2 - ]; tight)] + │ │ │ │ └── g:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ │ │ └── secondary: filters + │ │ │ └── c:14 > 1 [outer=(14), constraints=(/14: [/2 - ]; tight)] + │ │ ├── key: (11) + │ │ └── fd: (11)-->(12-19) + │ └── filters + │ └── k:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + └── projections + ├── false [as=partial_index_put1:22] + ├── false [as=partial_index_put2:23] + ├── false [as=partial_index_put3:24] + └── 2 [as=a_new:21] + +# Simplify UPDATE partial index put/del column to false for second partial index +# only. +norm expect=SimplifyPartialIndexProjections +UPDATE t SET a = 2, d = 2 WHERE k = 1 +---- +update t + ├── columns: + ├── fetch columns: k:11 a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── update-mapping: + │ ├── a_new:21 => a:2 + │ └── a_new:21 => d:5 + ├── partial index put columns: partial_index_put1:22 partial_index_put2:24 partial_index_put3:25 + ├── partial index del columns: partial_index_del1:23 partial_index_put2:24 partial_index_put3:25 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:22!null partial_index_del1:23 partial_index_put2:24!null partial_index_put3:25 a_new:21!null k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(11-19,21-25) + ├── select + │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11-19) + │ ├── scan t + │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ ├── partial index predicates + │ │ │ ├── secondary: filters + │ │ │ │ └── d:15 > 1 [outer=(15), constraints=(/15: [/2 - ]; tight)] + │ │ │ ├── secondary: filters + │ │ │ │ ├── f:17 > 1 [outer=(17), constraints=(/17: [/2 - ]; tight)] + │ │ │ │ └── g:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ │ │ └── secondary: filters + │ │ │ └── c:14 > 1 [outer=(14), constraints=(/14: [/2 - ]; tight)] + │ │ ├── key: (11) + │ │ └── fd: (11)-->(12-19) + │ └── filters + │ └── k:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + └── projections + ├── true [as=partial_index_put1:22] + ├── d:15 > 1 [as=partial_index_del1:23, outer=(15)] + ├── false [as=partial_index_put2:24] + ├── c:14 > 1 [as=partial_index_put3:25, outer=(14)] + └── 2 [as=a_new:21] + +# Do not simplify partial index put/del column to false when the indexed columns +# are mutating. +norm expect-not=SimplifyPartialIndexProjections +UPDATE t SET c = 1, e = 1 WHERE k = 1 +---- +update t + ├── columns: + ├── fetch columns: k:11 a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── update-mapping: + │ ├── c_new:21 => c:4 + │ └── c_new:21 => e:6 + ├── partial index put columns: partial_index_put1:22 partial_index_put2:23 partial_index_put3:24 + ├── partial index del columns: partial_index_put1:22 partial_index_put2:23 partial_index_del3:25 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:22 partial_index_put2:23 partial_index_put3:24!null partial_index_del3:25 c_new:21!null k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(11-19,21-25) + ├── select + │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11-19) + │ ├── scan t + │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ ├── partial index predicates + │ │ │ ├── secondary: filters + │ │ │ │ └── d:15 > 1 [outer=(15), constraints=(/15: [/2 - ]; tight)] + │ │ │ ├── secondary: filters + │ │ │ │ ├── f:17 > 1 [outer=(17), constraints=(/17: [/2 - ]; tight)] + │ │ │ │ └── g:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ │ │ └── secondary: filters + │ │ │ └── c:14 > 1 [outer=(14), constraints=(/14: [/2 - ]; tight)] + │ │ ├── key: (11) + │ │ └── fd: (11)-->(12-19) + │ └── filters + │ └── k:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + └── projections + ├── d:15 > 1 [as=partial_index_put1:22, outer=(15)] + ├── (f:17 > 1) AND (g:18 > 1) [as=partial_index_put2:23, outer=(17,18)] + ├── false [as=partial_index_put3:24] + ├── c:14 > 1 [as=partial_index_del3:25, outer=(14)] + └── 1 [as=c_new:21] + +# Do not simplify partial index put/del column to false when the columns +# referenced in partial index predicates are mutating. +norm expect-not=SimplifyPartialIndexProjections +UPDATE t SET d = d + 1, g = g + 1 WHERE k = 1 +---- +update t + ├── columns: + ├── fetch columns: k:11 a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── update-mapping: + │ ├── d_new:21 => d:5 + │ └── g_new:22 => g:8 + ├── partial index put columns: partial_index_put1:23 partial_index_put2:25 partial_index_put3:27 + ├── partial index del columns: partial_index_del1:24 partial_index_del2:26 partial_index_put3:27 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put1:23 partial_index_del1:24 partial_index_put2:25 partial_index_del2:26 partial_index_put3:27 k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 d_new:21 g_new:22 + ├── cardinality: [0 - 1] + ├── immutable + ├── key: () + ├── fd: ()-->(11-19,21-27) + ├── project + │ ├── columns: d_new:21 g_new:22 k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ ├── cardinality: [0 - 1] + │ ├── immutable + │ ├── key: () + │ ├── fd: ()-->(11-19,21,22) + │ ├── select + │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(11-19) + │ │ ├── scan t + │ │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ │ ├── partial index predicates + │ │ │ │ ├── secondary: filters + │ │ │ │ │ └── d:15 > 1 [outer=(15), constraints=(/15: [/2 - ]; tight)] + │ │ │ │ ├── secondary: filters + │ │ │ │ │ ├── f:17 > 1 [outer=(17), constraints=(/17: [/2 - ]; tight)] + │ │ │ │ │ └── g:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ │ │ │ └── secondary: filters + │ │ │ │ └── c:14 > 1 [outer=(14), constraints=(/14: [/2 - ]; tight)] + │ │ │ ├── key: (11) + │ │ │ └── fd: (11)-->(12-19) + │ │ └── filters + │ │ └── k:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + │ └── projections + │ ├── d:15 + 1 [as=d_new:21, outer=(15), immutable] + │ └── g:18 + 1 [as=g_new:22, outer=(18), immutable] + └── projections + ├── d_new:21 > 1 [as=partial_index_put1:23, outer=(21)] + ├── d:15 > 1 [as=partial_index_del1:24, outer=(15)] + ├── (f:17 > 1) AND (g_new:22 > 1) [as=partial_index_put2:25, outer=(17,22)] + ├── (f:17 > 1) AND (g:18 > 1) [as=partial_index_del2:26, outer=(17,18)] + └── c:14 > 1 [as=partial_index_put3:27, outer=(14)] + +# Do not simplify partial index put/del column to false when it is also an +# update column (h_new). +norm +UPDATE t SET h = d > 1 WHERE k = 1 +---- +update t + ├── columns: + ├── fetch columns: k:11 a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── update-mapping: + │ └── h_new:21 => h:9 + ├── partial index put columns: h_new:21 partial_index_put2:22 partial_index_put3:23 + ├── partial index del columns: h_new:21 partial_index_put2:22 partial_index_put3:23 + ├── cardinality: [0 - 0] + ├── volatile, mutations + └── project + ├── columns: partial_index_put2:22!null partial_index_put3:23!null h_new:21 k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(11-19,21-23) + ├── select + │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11-19) + │ ├── scan t + │ │ ├── columns: k:11!null a:12 b:13 c:14 d:15 e:16 f:17 g:18 h:19 + │ │ ├── partial index predicates + │ │ │ ├── secondary: filters + │ │ │ │ └── d:15 > 1 [outer=(15), constraints=(/15: [/2 - ]; tight)] + │ │ │ ├── secondary: filters + │ │ │ │ ├── f:17 > 1 [outer=(17), constraints=(/17: [/2 - ]; tight)] + │ │ │ │ └── g:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ │ │ └── secondary: filters + │ │ │ └── c:14 > 1 [outer=(14), constraints=(/14: [/2 - ]; tight)] + │ │ ├── key: (11) + │ │ └── fd: (11)-->(12-19) + │ └── filters + │ └── k:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + └── projections + ├── false [as=partial_index_put2:22] + ├── false [as=partial_index_put3:23] + └── d:15 > 1 [as=h_new:21, outer=(15)] diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index ef54a006d307..d07165b60127 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -2076,27 +2076,27 @@ update partial_indexes ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: partial_index_put1:12 d_new:11 a:6!null d:9 + ├── columns: partial_index_put1:12!null d_new:11 a:6!null d:9 ├── cardinality: [0 - 1] ├── immutable ├── key: () ├── fd: ()-->(6,9,11,12) ├── select - │ ├── columns: a:6!null b:7 d:9 + │ ├── columns: a:6!null d:9 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6,7,9) + │ ├── fd: ()-->(6,9) │ ├── scan partial_indexes - │ │ ├── columns: a:6!null b:7 d:9 + │ │ ├── columns: a:6!null d:9 │ │ ├── partial index predicates │ │ │ └── secondary: filters │ │ │ └── b:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] │ │ ├── key: (6) - │ │ └── fd: (6)-->(7,9) + │ │ └── fd: (6)-->(9) │ └── filters │ └── a:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] └── projections - ├── b:7 > 1 [as=partial_index_put1:12, outer=(7)] + ├── false [as=partial_index_put1:12] └── d:9 + 1 [as=d_new:11, outer=(9), immutable] # Do not prune the indexed column c when a column in the partial index From c8ffd8676b69c075e01241d80ca215ffbe451e71 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 5 Jan 2021 16:26:52 -0800 Subject: [PATCH 07/10] opt: fix columns in SplitScanIntoUnionScans constraint This commit fixes a minor bug in `SplitScanIntoUnionScans` that resulted in a scan's constraint containing columns not associated with the scan. This did not affect the correctness of results. However it appears that it did cause inaccurate stats calculations; I had to add histogram buckets to the tests to coerce the optimizer into choosing the same plan for the corresponding test. Release note: None --- pkg/sql/opt/xform/limit_funcs.go | 2 +- pkg/sql/opt/xform/testdata/rules/limit | 22 ++++++++++++++++------ 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/pkg/sql/opt/xform/limit_funcs.go b/pkg/sql/opt/xform/limit_funcs.go index c2556d8ac399..3e7d59dc4670 100644 --- a/pkg/sql/opt/xform/limit_funcs.go +++ b/pkg/sql/opt/xform/limit_funcs.go @@ -302,7 +302,7 @@ func (c *CustomFuncs) SplitScanIntoUnionScans( // construct an unlimited Scan and add it to the Union tree. newScanPrivate := c.DuplicateScanPrivate(sp) newScanPrivate.Constraint = &constraint.Constraint{ - Columns: sp.Constraint.Columns, + Columns: sp.Constraint.Columns.RemapColumns(sp.Table, newScanPrivate.Table), Spans: noLimitSpans, } newScan := c.e.f.ConstructScan(newScanPrivate) diff --git a/pkg/sql/opt/xform/testdata/rules/limit b/pkg/sql/opt/xform/testdata/rules/limit index 5c84fce47435..205241365e71 100644 --- a/pkg/sql/opt/xform/testdata/rules/limit +++ b/pkg/sql/opt/xform/testdata/rules/limit @@ -65,6 +65,9 @@ CREATE TABLE partial_index_tab ) ---- +# Insert statistics for index_tab. Histogram buckets are included for the +# latitude column in order to make the optimizer choose specific plans for +# SplitScanIntoUnionScans tests. exec-ddl ALTER TABLE index_tab INJECT STATISTICS '[ { @@ -83,7 +86,14 @@ ALTER TABLE index_tab INJECT STATISTICS '[ "columns": ["latitude"], "created_at": "2018-01-01 2:00:00.00000+00:00", "row_count": 1000000, - "distinct_count": 100 + "distinct_count": 100, + "null_count": 0, + "histo_col_type": "int", + "histo_buckets": [ + {"num_eq": 1000, "num_range": 0, "distinct_range": 0, "upper_bound": "-5"}, + {"num_eq": 1000, "num_range": 0, "distinct_range": 0, "upper_bound": "0"}, + {"num_eq": 0, "num_range": 998000, "distinct_range": 98, "upper_bound": "2000"} + ] }, { "columns": ["longitude"], @@ -1174,13 +1184,13 @@ limit ├── cardinality: [0 - 10] ├── ordering: +6,+7 ├── sort - │ ├── columns: latitude:4 longitude:5 data1:6!null data2:7!null + │ ├── columns: latitude:4!null longitude:5 data1:6!null data2:7!null │ ├── key: (4-7) │ ├── ordering: +6,+7 │ ├── limit hint: 10.00 │ └── union - │ ├── columns: latitude:4 longitude:5 data1:6!null data2:7!null - │ ├── left columns: latitude:4 longitude:5 data1:6!null data2:7!null + │ ├── columns: latitude:4!null longitude:5 data1:6!null data2:7!null + │ ├── left columns: latitude:4!null longitude:5 data1:6!null data2:7!null │ ├── right columns: latitude:64 longitude:65 data1:66 data2:67 │ ├── key: (4-7) │ ├── union @@ -1211,8 +1221,8 @@ limit │ │ ├── limit: 10 │ │ └── fd: ()-->(54,55) │ └── scan index_tab@d - │ ├── columns: latitude:64 longitude:65 data1:66!null data2:67!null - │ └── constraint: /4/5/6/7/1 + │ ├── columns: latitude:64!null longitude:65 data1:66!null data2:67!null + │ └── constraint: /64/65/66/67/61 │ ├── [/-5 - /-5] │ └── [/0 - /0] └── 10 From 58b3efb1b5516bbaf2916a360b98933d547180f3 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 5 Jan 2021 15:45:57 -0800 Subject: [PATCH 08/10] opt: do not reuse columns for Unions in SplitScanIntoUnionScans Unions generated in SplitScanIntoUnionScans no longer reuse column IDs from their left children as output column IDs. Reusing column IDs in this way has shown to be dangerous (see #58434). Release note: None --- pkg/sql/opt/norm/general_funcs.go | 38 ++++++---- pkg/sql/opt/xform/limit_funcs.go | 27 ++++--- pkg/sql/opt/xform/testdata/external/trading | 36 +++++----- .../xform/testdata/external/trading-mutation | 36 +++++----- pkg/sql/opt/xform/testdata/rules/limit | 70 +++++++++---------- 5 files changed, 113 insertions(+), 94 deletions(-) diff --git a/pkg/sql/opt/norm/general_funcs.go b/pkg/sql/opt/norm/general_funcs.go index 2699af28179e..57a637affc79 100644 --- a/pkg/sql/opt/norm/general_funcs.go +++ b/pkg/sql/opt/norm/general_funcs.go @@ -290,6 +290,26 @@ func (c *CustomFuncs) RedundantCols(input memo.RelExpr, cols opt.ColSet) opt.Col return cols.Difference(reducedCols) } +// DuplicateColumnIDs duplicates a table and set of columns IDs in the metadata. +// It returns the new table's ID and the new set of columns IDs. +func (c *CustomFuncs) DuplicateColumnIDs( + table opt.TableID, cols opt.ColSet, +) (opt.TableID, opt.ColSet) { + md := c.mem.Metadata() + tabMeta := md.TableMeta(table) + newTableID := md.DuplicateTable(table, c.RemapCols) + + // Build a new set of column IDs from the new TableMeta. + var newColIDs opt.ColSet + for col, ok := cols.Next(0); ok; col, ok = cols.Next(col + 1) { + ord := tabMeta.MetaID.ColumnOrdinal(col) + newColID := newTableID.ColumnID(ord) + newColIDs.Add(newColID) + } + + return newTableID, newColIDs +} + // RemapCols remaps columns IDs in the input ScalarExpr by replacing occurrences // of the keys of colMap with the corresponding values. If column IDs are // encountered in the input ScalarExpr that are not keys in colMap, they are not @@ -1054,23 +1074,11 @@ func (c *CustomFuncs) DatumsEqual(first, second tree.Datum) bool { // ScanPrivate, so the new ScanPrivate will not have constraints even if the old // one did. func (c *CustomFuncs) DuplicateScanPrivate(sp *memo.ScanPrivate) *memo.ScanPrivate { - md := c.mem.Metadata() - tabMeta := md.TableMeta(sp.Table) - newTableID := md.DuplicateTable(sp.Table, c.RemapCols) - - // Build a new set of column IDs from the new TableMeta. - var newColIDs opt.ColSet - cols := sp.Cols - for col, ok := cols.Next(0); ok; col, ok = cols.Next(col + 1) { - ord := tabMeta.MetaID.ColumnOrdinal(col) - newColID := newTableID.ColumnID(ord) - newColIDs.Add(newColID) - } - + table, cols := c.DuplicateColumnIDs(sp.Table, sp.Cols) return &memo.ScanPrivate{ - Table: newTableID, + Table: table, Index: sp.Index, - Cols: newColIDs, + Cols: cols, Flags: sp.Flags, Locking: sp.Locking, } diff --git a/pkg/sql/opt/xform/limit_funcs.go b/pkg/sql/opt/xform/limit_funcs.go index 3e7d59dc4670..53e8f32411d7 100644 --- a/pkg/sql/opt/xform/limit_funcs.go +++ b/pkg/sql/opt/xform/limit_funcs.go @@ -249,12 +249,11 @@ func (c *CustomFuncs) SplitScanIntoUnionScans( // makeNewUnion extends the Union tree rooted at 'last' to include 'newScan'. // The ColumnIDs of the original Scan are used by the resulting expression. - oldColList := scan.Relational().OutputCols.ToList() - makeNewUnion := func(last, newScan memo.RelExpr) memo.RelExpr { + makeNewUnion := func(last, newScan memo.RelExpr, outCols opt.ColList) memo.RelExpr { return c.e.f.ConstructUnion(last, newScan, &memo.SetPrivate{ LeftCols: last.Relational().OutputCols.ToList(), RightCols: newScan.Relational().OutputCols.ToList(), - OutCols: oldColList, + OutCols: outCols, }) } @@ -263,7 +262,7 @@ func (c *CustomFuncs) SplitScanIntoUnionScans( // construct a single unlimited Scan, which will also be added to the Unions. var noLimitSpans constraint.Spans var last memo.RelExpr - for i := 0; i < spans.Count(); i++ { + for i, n := 0, spans.Count(); i < n; i++ { if i >= budgetExceededIndex { // The Scan budget has been reached; no additional Scans can be created. noLimitSpans.Append(spans.Get(i)) @@ -276,15 +275,27 @@ func (c *CustomFuncs) SplitScanIntoUnionScans( noLimitSpans.Append(spans.Get(i)) continue } - for j := 0; j < singleKeySpans.Count(); j++ { + for j, m := 0, singleKeySpans.Count(); j < m; j++ { if last == nil { // This is the first limited Scan, so no Union necessary. last = c.makeNewScan(sp, cons.Columns, newHardLimit, singleKeySpans.Get(j)) continue } - // Construct a new Scan for each span and add it to the Union tree. + // Construct a new Scan for each span. newScan := c.makeNewScan(sp, cons.Columns, newHardLimit, singleKeySpans.Get(j)) - last = makeNewUnion(last, newScan) + + // Add the scan to the union tree. If it is the final union in the + // tree, use the original scan's columns as the union's out columns. + // Otherwise, create new output column IDs for the union. + var outCols opt.ColList + finalUnion := i == n-1 && j == m-1 && noLimitSpans.Count() == 0 + if finalUnion { + outCols = sp.Cols.ToList() + } else { + _, cols := c.DuplicateColumnIDs(sp.Table, sp.Cols) + outCols = cols.ToList() + } + last = makeNewUnion(last, newScan, outCols) } } if noLimitSpans.Count() == spans.Count() { @@ -306,7 +317,7 @@ func (c *CustomFuncs) SplitScanIntoUnionScans( Spans: noLimitSpans, } newScan := c.e.f.ConstructScan(newScanPrivate) - return makeNewUnion(last, newScan) + return makeNewUnion(last, newScan, sp.Cols.ToList()) } // indexHasOrderingSequence returns whether the Scan can provide a given diff --git a/pkg/sql/opt/xform/testdata/external/trading b/pkg/sql/opt/xform/testdata/external/trading index 1a7770a32315..a9d219550f2a 100644 --- a/pkg/sql/opt/xform/testdata/external/trading +++ b/pkg/sql/opt/xform/testdata/external/trading @@ -612,25 +612,25 @@ project │ │ │ ├── limit hint: 1.00 │ │ │ └── union │ │ │ ├── columns: dealerid:8!null version:16!null - │ │ │ ├── left columns: dealerid:8!null version:16!null - │ │ │ ├── right columns: dealerid:65 version:73 + │ │ │ ├── left columns: dealerid:75 version:83 + │ │ │ ├── right columns: dealerid:85 version:93 │ │ │ ├── cardinality: [0 - 4] │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] │ │ │ ├── key: (8,16) │ │ │ ├── union - │ │ │ │ ├── columns: dealerid:8!null version:16!null - │ │ │ │ ├── left columns: dealerid:8!null version:16!null - │ │ │ │ ├── right columns: dealerid:55 version:63 + │ │ │ │ ├── columns: dealerid:75!null version:83!null + │ │ │ │ ├── left columns: dealerid:55 version:63 + │ │ │ │ ├── right columns: dealerid:65 version:73 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── stats: [rows=3, distinct(8,16)=3, null(8,16)=0] - │ │ │ │ ├── key: (8,16) + │ │ │ │ ├── stats: [rows=3, distinct(75,83)=3, null(75,83)=0] + │ │ │ │ ├── key: (75,83) │ │ │ │ ├── union - │ │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ │ ├── columns: dealerid:55!null version:63!null │ │ │ │ │ ├── left columns: dealerid:35 version:43 │ │ │ │ │ ├── right columns: dealerid:45 version:53 │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── stats: [rows=2, distinct(8,16)=2, null(8,16)=0] - │ │ │ │ │ ├── key: (8,16) + │ │ │ │ │ ├── stats: [rows=2, distinct(55,63)=2, null(55,63)=0] + │ │ │ │ │ ├── key: (55,63) │ │ │ │ │ ├── scan cardsinfo@cardsinfoversionindex,rev │ │ │ │ │ │ ├── columns: dealerid:35!null version:43!null │ │ │ │ │ │ ├── constraint: /35/43: [/1 - /1] @@ -646,19 +646,19 @@ project │ │ │ │ │ ├── key: () │ │ │ │ │ └── fd: ()-->(45,53) │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ ├── columns: dealerid:55!null version:63!null - │ │ │ │ ├── constraint: /55/63: [/3 - /3] + │ │ │ │ ├── columns: dealerid:65!null version:73!null + │ │ │ │ ├── constraint: /65/73: [/3 - /3] │ │ │ │ ├── limit: 1(rev) - │ │ │ │ ├── stats: [rows=1, distinct(55)=1, null(55)=0, distinct(55,63)=1, null(55,63)=0] + │ │ │ │ ├── stats: [rows=1, distinct(65)=1, null(65)=0, distinct(65,73)=1, null(65,73)=0] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(55,63) + │ │ │ │ └── fd: ()-->(65,73) │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ ├── columns: dealerid:65!null version:73!null - │ │ │ ├── constraint: /65/73: [/4 - /4] + │ │ │ ├── columns: dealerid:85!null version:93!null + │ │ │ ├── constraint: /85/93: [/4 - /4] │ │ │ ├── limit: 1(rev) - │ │ │ ├── stats: [rows=1, distinct(65)=1, null(65)=0, distinct(65,73)=1, null(65,73)=0] + │ │ │ ├── stats: [rows=1, distinct(85)=1, null(85)=0, distinct(85,93)=1, null(85,93)=0] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(65,73) + │ │ │ └── fd: ()-->(85,93) │ │ └── 1 │ └── aggregations │ └── const-agg [as=max:33, outer=(16)] diff --git a/pkg/sql/opt/xform/testdata/external/trading-mutation b/pkg/sql/opt/xform/testdata/external/trading-mutation index 7641f09129d8..66d8ef3defb8 100644 --- a/pkg/sql/opt/xform/testdata/external/trading-mutation +++ b/pkg/sql/opt/xform/testdata/external/trading-mutation @@ -618,25 +618,25 @@ project │ │ │ ├── limit hint: 1.00 │ │ │ └── union │ │ │ ├── columns: dealerid:8!null version:16!null - │ │ │ ├── left columns: dealerid:8!null version:16!null - │ │ │ ├── right columns: dealerid:81 version:89 + │ │ │ ├── left columns: dealerid:95 version:103 + │ │ │ ├── right columns: dealerid:109 version:117 │ │ │ ├── cardinality: [0 - 4] │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] │ │ │ ├── key: (8,16) │ │ │ ├── union - │ │ │ │ ├── columns: dealerid:8!null version:16!null - │ │ │ │ ├── left columns: dealerid:8!null version:16!null - │ │ │ │ ├── right columns: dealerid:67 version:75 + │ │ │ │ ├── columns: dealerid:95!null version:103!null + │ │ │ │ ├── left columns: dealerid:67 version:75 + │ │ │ │ ├── right columns: dealerid:81 version:89 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── stats: [rows=3, distinct(8,16)=3, null(8,16)=0] - │ │ │ │ ├── key: (8,16) + │ │ │ │ ├── stats: [rows=3, distinct(95,103)=3, null(95,103)=0] + │ │ │ │ ├── key: (95,103) │ │ │ │ ├── union - │ │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ │ ├── columns: dealerid:67!null version:75!null │ │ │ │ │ ├── left columns: dealerid:39 version:47 │ │ │ │ │ ├── right columns: dealerid:53 version:61 │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── stats: [rows=2, distinct(8,16)=2, null(8,16)=0] - │ │ │ │ │ ├── key: (8,16) + │ │ │ │ │ ├── stats: [rows=2, distinct(67,75)=2, null(67,75)=0] + │ │ │ │ │ ├── key: (67,75) │ │ │ │ │ ├── scan cardsinfo@cardsinfoversionindex,rev │ │ │ │ │ │ ├── columns: dealerid:39!null version:47!null │ │ │ │ │ │ ├── constraint: /39/47: [/1 - /1] @@ -652,19 +652,19 @@ project │ │ │ │ │ ├── key: () │ │ │ │ │ └── fd: ()-->(53,61) │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ ├── columns: dealerid:67!null version:75!null - │ │ │ │ ├── constraint: /67/75: [/3 - /3] + │ │ │ │ ├── columns: dealerid:81!null version:89!null + │ │ │ │ ├── constraint: /81/89: [/3 - /3] │ │ │ │ ├── limit: 1(rev) - │ │ │ │ ├── stats: [rows=1, distinct(67)=1, null(67)=0, distinct(67,75)=1, null(67,75)=0] + │ │ │ │ ├── stats: [rows=1, distinct(81)=1, null(81)=0, distinct(81,89)=1, null(81,89)=0] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(67,75) + │ │ │ │ └── fd: ()-->(81,89) │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ ├── columns: dealerid:81!null version:89!null - │ │ │ ├── constraint: /81/89: [/4 - /4] + │ │ │ ├── columns: dealerid:109!null version:117!null + │ │ │ ├── constraint: /109/117: [/4 - /4] │ │ │ ├── limit: 1(rev) - │ │ │ ├── stats: [rows=1, distinct(81)=1, null(81)=0, distinct(81,89)=1, null(81,89)=0] + │ │ │ ├── stats: [rows=1, distinct(109)=1, null(109)=0, distinct(109,117)=1, null(109,117)=0] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(81,89) + │ │ │ └── fd: ()-->(109,117) │ │ └── 1 │ └── aggregations │ └── const-agg [as=max:37, outer=(16)] diff --git a/pkg/sql/opt/xform/testdata/rules/limit b/pkg/sql/opt/xform/testdata/rules/limit index 205241365e71..9bd2b92cf03f 100644 --- a/pkg/sql/opt/xform/testdata/rules/limit +++ b/pkg/sql/opt/xform/testdata/rules/limit @@ -842,16 +842,16 @@ limit │ ├── limit hint: 10.00 │ └── union │ ├── columns: val:2!null data1:6!null - │ ├── left columns: val:2!null data1:6!null - │ ├── right columns: val:32 data1:36 + │ ├── left columns: val:32 data1:36 + │ ├── right columns: val:42 data1:46 │ ├── cardinality: [0 - 30] │ ├── key: (2,6) │ ├── union - │ │ ├── columns: val:2!null data1:6!null + │ │ ├── columns: val:32!null data1:36!null │ │ ├── left columns: val:12 data1:16 │ │ ├── right columns: val:22 data1:26 │ │ ├── cardinality: [0 - 20] - │ │ ├── key: (2,6) + │ │ ├── key: (32,36) │ │ ├── scan index_tab@b │ │ │ ├── columns: val:12!null data1:16!null │ │ │ ├── constraint: /12/16/17/11: [/1 - /1] @@ -863,10 +863,10 @@ limit │ │ ├── limit: 10 │ │ └── fd: ()-->(22) │ └── scan index_tab@b - │ ├── columns: val:32!null data1:36!null - │ ├── constraint: /32/36/37/31: [/3 - /3] + │ ├── columns: val:42!null data1:46!null + │ ├── constraint: /42/46/47/41: [/3 - /3] │ ├── limit: 10 - │ └── fd: ()-->(32) + │ └── fd: ()-->(42) └── 10 # Case with single-key spans. @@ -982,16 +982,16 @@ scalar-group-by │ │ ├── limit hint: 1.00 │ │ └── union │ │ ├── columns: val:2!null data1:6!null - │ │ ├── left columns: val:2!null data1:6!null - │ │ ├── right columns: val:33 data1:37 + │ │ ├── left columns: val:33 data1:37 + │ │ ├── right columns: val:43 data1:47 │ │ ├── cardinality: [0 - 3] │ │ ├── key: (2,6) │ │ ├── union - │ │ │ ├── columns: val:2!null data1:6!null + │ │ │ ├── columns: val:33!null data1:37!null │ │ │ ├── left columns: val:13 data1:17 │ │ │ ├── right columns: val:23 data1:27 │ │ │ ├── cardinality: [0 - 2] - │ │ │ ├── key: (2,6) + │ │ │ ├── key: (33,37) │ │ │ ├── scan index_tab@b,rev │ │ │ │ ├── columns: val:13!null data1:17!null │ │ │ │ ├── constraint: /13/17/18/12: [/1 - /1] @@ -1005,11 +1005,11 @@ scalar-group-by │ │ │ ├── key: () │ │ │ └── fd: ()-->(23,27) │ │ └── scan index_tab@b,rev - │ │ ├── columns: val:33!null data1:37!null - │ │ ├── constraint: /33/37/38/32: [/3 - /3] + │ │ ├── columns: val:43!null data1:47!null + │ │ ├── constraint: /43/47/48/42: [/3 - /3] │ │ ├── limit: 1(rev) │ │ ├── key: () - │ │ └── fd: ()-->(33,37) + │ │ └── fd: ()-->(43,47) │ └── 1 └── aggregations └── const-agg [as=max:11, outer=(6)] @@ -1190,21 +1190,21 @@ limit │ ├── limit hint: 10.00 │ └── union │ ├── columns: latitude:4!null longitude:5 data1:6!null data2:7!null - │ ├── left columns: latitude:4!null longitude:5 data1:6!null data2:7!null - │ ├── right columns: latitude:64 longitude:65 data1:66 data2:67 + │ ├── left columns: latitude:74 longitude:75 data1:76 data2:77 + │ ├── right columns: latitude:84 longitude:85 data1:86 data2:87 │ ├── key: (4-7) │ ├── union - │ │ ├── columns: latitude:4!null longitude:5!null data1:6!null data2:7!null - │ │ ├── left columns: latitude:4!null longitude:5!null data1:6!null data2:7!null - │ │ ├── right columns: latitude:54 longitude:55 data1:56 data2:57 + │ │ ├── columns: latitude:74!null longitude:75!null data1:76!null data2:77!null + │ │ ├── left columns: latitude:54 longitude:55 data1:56 data2:57 + │ │ ├── right columns: latitude:64 longitude:65 data1:66 data2:67 │ │ ├── cardinality: [0 - 30] - │ │ ├── key: (4-7) + │ │ ├── key: (74-77) │ │ ├── union - │ │ │ ├── columns: latitude:4!null longitude:5!null data1:6!null data2:7!null + │ │ │ ├── columns: latitude:54!null longitude:55!null data1:56!null data2:57!null │ │ │ ├── left columns: latitude:34 longitude:35 data1:36 data2:37 │ │ │ ├── right columns: latitude:44 longitude:45 data1:46 data2:47 │ │ │ ├── cardinality: [0 - 20] - │ │ │ ├── key: (4-7) + │ │ │ ├── key: (54-57) │ │ │ ├── scan index_tab@d │ │ │ │ ├── columns: latitude:34!null longitude:35!null data1:36!null data2:37!null │ │ │ │ ├── constraint: /34/35/36/37/31: [/10/11 - /10/11] @@ -1216,13 +1216,13 @@ limit │ │ │ ├── limit: 10 │ │ │ └── fd: ()-->(44,45) │ │ └── scan index_tab@d - │ │ ├── columns: latitude:54!null longitude:55!null data1:56!null data2:57!null - │ │ ├── constraint: /54/55/56/57/51: [/10/13 - /10/13] + │ │ ├── columns: latitude:64!null longitude:65!null data1:66!null data2:67!null + │ │ ├── constraint: /64/65/66/67/61: [/10/13 - /10/13] │ │ ├── limit: 10 - │ │ └── fd: ()-->(54,55) + │ │ └── fd: ()-->(64,65) │ └── scan index_tab@d - │ ├── columns: latitude:64!null longitude:65 data1:66!null data2:67!null - │ └── constraint: /64/65/66/67/61 + │ ├── columns: latitude:84!null longitude:85 data1:86!null data2:87!null + │ └── constraint: /84/85/86/87/81 │ ├── [/-5 - /-5] │ └── [/0 - /0] └── 10 @@ -1291,16 +1291,16 @@ limit │ ├── limit hint: 5.00 │ └── union │ ├── columns: p:1!null q:2!null r:3!null s:4!null - │ ├── left columns: p:1!null q:2!null r:3!null s:4!null - │ ├── right columns: p:16 q:17 r:18 s:19 + │ ├── left columns: p:16 q:17 r:18 s:19 + │ ├── right columns: p:21 q:22 r:23 s:24 │ ├── cardinality: [0 - 15] │ ├── key: (1-4) │ ├── union - │ │ ├── columns: p:1!null q:2!null r:3!null s:4!null + │ │ ├── columns: p:16!null q:17!null r:18!null s:19!null │ │ ├── left columns: p:6 q:7 r:8 s:9 │ │ ├── right columns: p:11 q:12 r:13 s:14 │ │ ├── cardinality: [0 - 10] - │ │ ├── key: (1-4) + │ │ ├── key: (16-19) │ │ ├── scan pqrs │ │ │ ├── columns: p:6!null q:7!null r:8!null s:9!null │ │ │ ├── constraint: /6/7: [/1 - /1] @@ -1314,11 +1314,11 @@ limit │ │ ├── key: (12) │ │ └── fd: ()-->(11), (12)-->(13,14) │ └── scan pqrs - │ ├── columns: p:16!null q:17!null r:18!null s:19!null - │ ├── constraint: /16/17: [/10 - /10] + │ ├── columns: p:21!null q:22!null r:23!null s:24!null + │ ├── constraint: /21/22: [/10 - /10] │ ├── limit: 5 - │ ├── key: (17) - │ └── fd: ()-->(16), (17)-->(18,19) + │ ├── key: (22) + │ └── fd: ()-->(21), (22)-->(23,24) └── 5 # Case where multiple check constraints are combined into one constraint From 80d76052d6c1628271fd01b1c2deb6dc2bb5d011 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 5 Jan 2021 16:39:48 -0800 Subject: [PATCH 09/10] opt: add Union column ID check to CheckExpr A check has been added to `CheckExpr` that asserts that the output columns of `Union`s and `UnionAll`s are not reused from the left or right inputs of the union. Reusing columns in this way is dangerous (see #58434). Release note: None --- pkg/sql/opt/memo/check_expr.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/pkg/sql/opt/memo/check_expr.go b/pkg/sql/opt/memo/check_expr.go index c45f26a2c1c4..1f9096150f76 100644 --- a/pkg/sql/opt/memo/check_expr.go +++ b/pkg/sql/opt/memo/check_expr.go @@ -103,6 +103,30 @@ func (m *Memo) CheckExpr(e opt.Expr) { case *SelectExpr: checkFilters(t.Filters) + case *UnionExpr, *UnionAllExpr: + setPrivate := t.Private().(*SetPrivate) + outColSet := setPrivate.OutCols.ToSet() + + // Check that columns on the left side of the union are not reused in + // the output. + leftColSet := setPrivate.LeftCols.ToSet() + if outColSet.Intersects(leftColSet) { + panic(errors.AssertionFailedf( + "union reuses columns in left input: %v", + outColSet.Intersection(leftColSet), + )) + } + + // Check that columns on the right side of the union are not reused in + // the output. + rightColSet := setPrivate.RightCols.ToSet() + if outColSet.Intersects(rightColSet) { + panic(errors.AssertionFailedf( + "union reuses columns in right input: %v", + outColSet.Intersection(rightColSet), + )) + } + case *AggregationsExpr: var checkAggs func(scalar opt.ScalarExpr) checkAggs = func(scalar opt.ScalarExpr) { From 5797332b012da21d287efa143b77d22f76d8fb1b Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Wed, 30 Dec 2020 10:26:19 -0500 Subject: [PATCH 10/10] streamingccl: add ingestion job framework This change introduces a new StreamIngestionJob. It does not do much more than laying out the general outline of the job, which is very similar to other bulk jobs such as changefeed, backup etc. More precisely: - Introduces StreamIngestionDetails job details proto - Hooks up the dependancy to a mock stream client - Introduces a StreamIngestionProcessorSpec - Sets up a simple DistSQL flow which round robin assigns the partitions to the processors. Most notable TODOs in job land which will be addressed in follow up PRs: - StreamIngestionPlanHook to create this job. Will involve figuring out SQL syntax. - Introducing a ts watermark in both the job and processors. This watermark will represent the lowest resolved ts which all processors have ingested till. Iron out semantics on job start and resumption. - Introducing a StreamIngestionFrontier processor which will slurp the results from the StreamIngestionProcessors, and use them to keep track of the minimum resolved ts across all processors. Release note: None --- pkg/ccl/streamingccl/BUILD.bazel | 19 +- pkg/ccl/streamingccl/stream_ingestion_job.go | 85 +- .../stream_ingestion_processor_planning.go | 109 +++ pkg/ccl/streamingccl/streamclient/BUILD.bazel | 1 + .../streamingccl/streamclient/client_test.go | 10 +- .../streamclient/stream_client.go | 33 + pkg/jobs/jobspb/BUILD.bazel | 1 + pkg/jobs/jobspb/jobs.pb.go | 823 +++++++++++------- pkg/jobs/jobspb/jobs.proto | 6 + pkg/jobs/jobspb/wrap.go | 2 +- pkg/sql/execinfrapb/BUILD.bazel | 1 + pkg/sql/execinfrapb/processors.pb.go | 227 +++-- pkg/sql/execinfrapb/processors.proto | 1 + pkg/sql/execinfrapb/processors_bulk_io.pb.go | 413 ++++++--- pkg/sql/execinfrapb/processors_bulk_io.proto | 4 + pkg/ts/catalog/chart_catalog.go | 12 + 16 files changed, 1200 insertions(+), 547 deletions(-) create mode 100644 pkg/ccl/streamingccl/stream_ingestion_processor_planning.go create mode 100644 pkg/ccl/streamingccl/streamclient/stream_client.go diff --git a/pkg/ccl/streamingccl/BUILD.bazel b/pkg/ccl/streamingccl/BUILD.bazel index 655fb40643ab..a47c426b7088 100644 --- a/pkg/ccl/streamingccl/BUILD.bazel +++ b/pkg/ccl/streamingccl/BUILD.bazel @@ -2,7 +2,24 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "streamingccl", - srcs = ["stream_ingestion_job.go"], + srcs = [ + "stream_ingestion_job.go", + "stream_ingestion_processor_planning.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl", visibility = ["//visibility:public"], + deps = [ + "//pkg/ccl/streamingccl/streamclient", + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/execinfrapb", + "//pkg/sql/physicalplan", + "//pkg/sql/sem/tree", + "//pkg/sql/types", + "@com_github_cockroachdb_logtags//:logtags", + ], ) diff --git a/pkg/ccl/streamingccl/stream_ingestion_job.go b/pkg/ccl/streamingccl/stream_ingestion_job.go index c1ee4d2eb72e..e930aad3d806 100644 --- a/pkg/ccl/streamingccl/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/stream_ingestion_job.go @@ -8,6 +8,89 @@ package streamingccl +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +type streamIngestionResumer struct { + job *jobs.Job +} + +func ingest( + ctx context.Context, + execCtx sql.JobExecContext, + streamAddress streamclient.PartitionAddress, + job *jobs.Job, +) error { + // Initialize a stream client and resolve topology. + client := streamclient.NewStreamClient() + sa := streamclient.StreamAddress(streamAddress) + topology, err := client.GetTopology(sa) + if err != nil { + return err + } + + evalCtx := execCtx.ExtendedEvalContext() + dsp := execCtx.DistSQLPlanner() + + planCtx, nodes, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) + if err != nil { + return err + } + + // Construct stream ingestion processor specs. + streamIngestionSpecs, err := distStreamIngestionPlanSpecs(topology, nodes) + if err != nil { + return err + } + + // Plan and run the DistSQL flow. + err = distStreamIngest(ctx, execCtx, nodes, planCtx, dsp, streamIngestionSpecs) + if err != nil { + return err + } + + return nil +} + +// Resume is part of the jobs.Resumer interface. +func (s *streamIngestionResumer) Resume( + ctx context.Context, execCtx interface{}, resultsCh chan<- tree.Datums, +) error { + details := s.job.Details().(jobspb.StreamIngestionDetails) + p := execCtx.(sql.JobExecContext) + + err := ingest(ctx, p, streamclient.PartitionAddress(details.StreamAddress), s.job) + if err != nil { + return err + } + + // TODO(adityamaru): We probably want to use the resultsCh to indicate that + // the processors have completed setup. We can then return the job ID in the + // plan hook similar to how changefeeds do it. + + return nil +} + +// OnFailOrCancel is part of the jobs.Resumer interface. +func (s *streamIngestionResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) error { + return nil +} + +var _ jobs.Resumer = &streamIngestionResumer{} + func init() { - // TODO: Implement me. + jobs.RegisterConstructor( + jobspb.TypeStreamIngestion, + func(job *jobs.Job, + settings *cluster.Settings) jobs.Resumer { + return &streamIngestionResumer{job: job} + }) } diff --git a/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go b/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go new file mode 100644 index 000000000000..7d4c138a91e7 --- /dev/null +++ b/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go @@ -0,0 +1,109 @@ +// Copyright 2020 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package streamingccl + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/logtags" +) + +// TODO(adityamaru): Figure out what the processors will return. +var streamIngestionResultTypes = []*types.T{} + +func distStreamIngestionPlanSpecs( + topology streamclient.Topology, nodes []roachpb.NodeID, +) ([]*execinfrapb.StreamIngestionDataSpec, error) { + + // For each stream partition in the topology, assign it to a node. + streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(nodes)) + + for i, partition := range topology.Partitions { + // Round robin assign the stream partitions to nodes. Partitions 0 through + // len(nodes) - 1 creates the spec. Future partitions just add themselves to + // the partition addresses. + if i < len(nodes) { + spec := &execinfrapb.StreamIngestionDataSpec{ + PartitionAddress: make([]streamclient.PartitionAddress, 0), + } + streamIngestionSpecs = append(streamIngestionSpecs, spec) + } + n := i % len(nodes) + streamIngestionSpecs[n].PartitionAddress = append(streamIngestionSpecs[n].PartitionAddress, partition) + } + + return streamIngestionSpecs, nil +} + +func distStreamIngest( + ctx context.Context, + execCtx sql.JobExecContext, + nodes []roachpb.NodeID, + planCtx *sql.PlanningCtx, + dsp *sql.DistSQLPlanner, + streamIngestionSpecs []*execinfrapb.StreamIngestionDataSpec, +) error { + ctx = logtags.AddTag(ctx, "stream-ingest-distsql", nil) + evalCtx := execCtx.ExtendedEvalContext() + var noTxn *kv.Txn + + if len(streamIngestionSpecs) == 0 { + return nil + } + + // Setup a one-stage plan with one proc per input spec. + corePlacement := make([]physicalplan.ProcessorCorePlacement, len(streamIngestionSpecs)) + for i := range streamIngestionSpecs { + corePlacement[i].NodeID = nodes[i] + corePlacement[i].Core.StreamIngestionData = streamIngestionSpecs[i] + } + + p := planCtx.NewPhysicalPlan() + p.AddNoInputStage( + corePlacement, + execinfrapb.PostProcessSpec{}, + streamIngestionResultTypes, + execinfrapb.Ordering{}, + ) + + // TODO(adityamaru): It is likely that we will add a StreamIngestFrontier + // processor on the coordinator node. All the StreamIngestionProcessors will + // feed their results into this frontier. This is similar to the relationship + // between the ChangeAggregator and ChangeFrontier processors. The + // StreamIngestFrontier will be responsible for updating the job watermark + // with the min of the resolved ts outputted by all the processors. + + // TODO(adityamaru): Once result types are updated, add PlanToStreamColMap. + dsp.FinalizePlan(planCtx, p) + + recv := sql.MakeDistSQLReceiver( + ctx, + // TODO(adityamaru): Are there any results we want to surface to the user? + nil, /* resultWriter */ + tree.Rows, + nil, /* rangeCache */ + noTxn, + nil, /* clockUpdater */ + evalCtx.Tracing, + ) + defer recv.Release() + + // Copy the evalCtx, as dsp.Run() might change it. + evalCtxCopy := *evalCtx + dsp.Run(planCtx, noTxn, p, recv, &evalCtxCopy, nil /* finishedSetupFn */) + return nil +} diff --git a/pkg/ccl/streamingccl/streamclient/BUILD.bazel b/pkg/ccl/streamingccl/streamclient/BUILD.bazel index d1b6e4da4732..0ca9ddd0576c 100644 --- a/pkg/ccl/streamingccl/streamclient/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamclient/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "client.go", "event.go", + "stream_client.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient", visibility = ["//visibility:public"], diff --git a/pkg/ccl/streamingccl/streamclient/client_test.go b/pkg/ccl/streamingccl/streamclient/client_test.go index c5ebbe59684b..01d055d34d9c 100644 --- a/pkg/ccl/streamingccl/streamclient/client_test.go +++ b/pkg/ccl/streamingccl/streamclient/client_test.go @@ -18,12 +18,12 @@ import ( "github.com/stretchr/testify/require" ) -type mockStreamClient struct{} +type testStreamClient struct{} -var _ Client = mockStreamClient{} +var _ Client = testStreamClient{} // GetTopology implements the Client interface. -func (sc mockStreamClient) GetTopology(_ StreamAddress) (Topology, error) { +func (sc testStreamClient) GetTopology(_ StreamAddress) (Topology, error) { return Topology{Partitions: []PartitionAddress{ "s3://my_bucket/my_stream/partition_1", "s3://my_bucket/my_stream/partition_2", @@ -31,7 +31,7 @@ func (sc mockStreamClient) GetTopology(_ StreamAddress) (Topology, error) { } // ConsumePartition implements the Client interface. -func (sc mockStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (chan Event, error) { +func (sc testStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (chan Event, error) { sampleKV := roachpb.KeyValue{ Key: []byte("key_1"), Value: roachpb.Value{ @@ -51,7 +51,7 @@ func (sc mockStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (ch // TestExampleClientUsage serves as documentation to indicate how a stream // client could be used. func TestExampleClientUsage(t *testing.T) { - client := mockStreamClient{} + client := testStreamClient{} sa := StreamAddress("s3://my_bucket/my_stream") topology, err := client.GetTopology(sa) require.NoError(t, err) diff --git a/pkg/ccl/streamingccl/streamclient/stream_client.go b/pkg/ccl/streamingccl/streamclient/stream_client.go new file mode 100644 index 000000000000..72a971953091 --- /dev/null +++ b/pkg/ccl/streamingccl/streamclient/stream_client.go @@ -0,0 +1,33 @@ +// Copyright 2020 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package streamclient + +import "time" + +// client is a mock stream client. +type client struct{} + +var _ Client = &client{} + +// NewStreamClient returns a new mock stream client. +func NewStreamClient() Client { + return &client{} +} + +// GetTopology implements the Client interface. +func (m *client) GetTopology(address StreamAddress) (Topology, error) { + panic("unimplemented mock method") +} + +// ConsumePartition implements the Client interface. +func (m *client) ConsumePartition( + address PartitionAddress, startTime time.Time, +) (chan Event, error) { + panic("unimplemented mock method") +} diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 87b1fd67bffc..879a13e4ff57 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobspb", visibility = ["//visibility:public"], deps = [ + "//pkg/ccl/streamingccl/streamclient", "//pkg/roachpb", "//pkg/security", "//pkg/sql/catalog/descpb", diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 22096fa0178b..6eefa434c85f 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -12,6 +12,7 @@ import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import github_com_cockroachdb_cockroach_pkg_sql_sem_tree "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -55,7 +56,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{0} } type Status int32 @@ -84,7 +85,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1} } type Type int32 @@ -102,19 +103,21 @@ const ( // We can't name this TYPE_SCHEMA_CHANGE due to how proto generates actual // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TypeTypeSchemaChange Type = 9 + TypeStreamIngestion Type = 10 ) var Type_name = map[int32]string{ - 0: "UNSPECIFIED", - 1: "BACKUP", - 2: "RESTORE", - 3: "SCHEMA_CHANGE", - 4: "IMPORT", - 5: "CHANGEFEED", - 6: "CREATE_STATS", - 7: "AUTO_CREATE_STATS", - 8: "SCHEMA_CHANGE_GC", - 9: "TYPEDESC_SCHEMA_CHANGE", + 0: "UNSPECIFIED", + 1: "BACKUP", + 2: "RESTORE", + 3: "SCHEMA_CHANGE", + 4: "IMPORT", + 5: "CHANGEFEED", + 6: "CREATE_STATS", + 7: "AUTO_CREATE_STATS", + 8: "SCHEMA_CHANGE_GC", + 9: "TYPEDESC_SCHEMA_CHANGE", + 10: "STREAM_INGESTION", } var Type_value = map[string]int32{ "UNSPECIFIED": 0, @@ -127,10 +130,11 @@ var Type_value = map[string]int32{ "AUTO_CREATE_STATS": 7, "SCHEMA_CHANGE_GC": 8, "TYPEDESC_SCHEMA_CHANGE": 9, + "STREAM_INGESTION": 10, } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2} } type EncryptionInfo_Scheme int32 @@ -150,7 +154,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -180,7 +184,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 0} } type Lease struct { @@ -194,7 +198,7 @@ func (m *Lease) Reset() { *m = Lease{} } func (m *Lease) String() string { return proto.CompactTextString(m) } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -235,7 +239,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions) ProtoMessage() {} func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -269,7 +273,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {} func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +313,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} } func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) } func (*EncryptionInfo) ProtoMessage() {} func (*EncryptionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -334,6 +338,41 @@ func (m *EncryptionInfo) XXX_DiscardUnknown() { var xxx_messageInfo_EncryptionInfo proto.InternalMessageInfo +type StreamIngestionDetails struct { + // StreamAddress is the location of the stream which the ingestion job will + // read from. + StreamAddress github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.StreamAddress `protobuf:"bytes,1,opt,name=stream_address,json=streamAddress,proto3,casttype=github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.StreamAddress" json:"stream_address,omitempty"` +} + +func (m *StreamIngestionDetails) Reset() { *m = StreamIngestionDetails{} } +func (m *StreamIngestionDetails) String() string { return proto.CompactTextString(m) } +func (*StreamIngestionDetails) ProtoMessage() {} +func (*StreamIngestionDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{3} +} +func (m *StreamIngestionDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StreamIngestionDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionDetails.Merge(dst, src) +} +func (m *StreamIngestionDetails) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionDetails) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionDetails proto.InternalMessageInfo + type BackupDetails struct { StartTime hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime,proto3" json:"start_time"` EndTime hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime,proto3" json:"end_time"` @@ -364,7 +403,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} } func (m *BackupDetails) String() string { return proto.CompactTextString(m) } func (*BackupDetails) ProtoMessage() {} func (*BackupDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{3} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{4} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -396,7 +435,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} } func (m *BackupProgress) String() string { return proto.CompactTextString(m) } func (*BackupProgress) ProtoMessage() {} func (*BackupProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{4} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{5} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +497,7 @@ func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } func (*RestoreDetails) ProtoMessage() {} func (*RestoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -495,7 +534,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {} func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -528,7 +567,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {} func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -561,7 +600,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } func (*RestoreProgress) ProtoMessage() {} func (*RestoreProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{6} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{7} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +661,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} } func (m *ImportDetails) String() string { return proto.CompactTextString(m) } func (*ImportDetails) ProtoMessage() {} func (*ImportDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{7} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{8} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -660,7 +699,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } func (*ImportDetails_Table) ProtoMessage() {} func (*ImportDetails_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{7, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{8, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -702,7 +741,7 @@ func (m *SequenceValChunk) Reset() { *m = SequenceValChunk{} } func (m *SequenceValChunk) String() string { return proto.CompactTextString(m) } func (*SequenceValChunk) ProtoMessage() {} func (*SequenceValChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{8} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{9} } func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -738,7 +777,7 @@ func (m *SequenceDetails) Reset() { *m = SequenceDetails{} } func (m *SequenceDetails) String() string { return proto.CompactTextString(m) } func (*SequenceDetails) ProtoMessage() {} func (*SequenceDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{9} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{10} } func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -773,7 +812,7 @@ func (m *SequenceDetails_SequenceChunks) Reset() { *m = SequenceDetails_ func (m *SequenceDetails_SequenceChunks) String() string { return proto.CompactTextString(m) } func (*SequenceDetails_SequenceChunks) ProtoMessage() {} func (*SequenceDetails_SequenceChunks) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{9, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{10, 0} } func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -819,7 +858,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{10} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{11} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -853,7 +892,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeDetails) ProtoMessage() {} func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{11} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{12} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -886,7 +925,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeProgress) ProtoMessage() {} func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{12} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{13} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -919,7 +958,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } func (*ResumeSpanList) ProtoMessage() {} func (*ResumeSpanList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{13} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{14} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -954,7 +993,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } func (*DroppedTableDetails) ProtoMessage() {} func (*DroppedTableDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{14} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{15} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1012,7 +1051,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} } func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails) ProtoMessage() {} func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1046,7 +1085,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1080,7 +1119,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1144,7 +1183,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeDetails) ProtoMessage() {} func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{16} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{17} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1176,7 +1215,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{17} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{18} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1212,7 +1251,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{} func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress) ProtoMessage() {} func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1246,7 +1285,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1280,7 +1319,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1352,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } func (*ChangefeedTarget) ProtoMessage() {} func (*ChangefeedTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{19} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{20} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1405,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} func (*ChangefeedDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{20} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{21} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1401,7 +1440,7 @@ func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} func (*ResolvedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{21} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{22} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1444,7 +1483,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} func (*ChangefeedProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{22} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{23} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1488,7 +1527,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} } func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails) ProtoMessage() {} func (*CreateStatsDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{23} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{24} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1529,7 +1568,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColStat) ProtoMessage() {} func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{23, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{24, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1561,7 +1600,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} } func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) } func (*CreateStatsProgress) ProtoMessage() {} func (*CreateStatsProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{24} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{25} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1628,7 +1667,7 @@ func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} func (*Payload) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{25} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{26} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1967,7 +2006,7 @@ func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} func (*Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{26} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{27} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2373,7 +2412,7 @@ func (m *Job) Reset() { *m = Job{} } func (m *Job) String() string { return proto.CompactTextString(m) } func (*Job) ProtoMessage() {} func (*Job) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{27} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{28} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2404,6 +2443,7 @@ func init() { proto.RegisterType((*BackupEncryptionOptions_KMSInfo)(nil), "cockroach.sql.jobs.jobspb.BackupEncryptionOptions.KMSInfo") proto.RegisterType((*EncryptionInfo)(nil), "cockroach.sql.jobs.jobspb.EncryptionInfo") proto.RegisterMapType((map[string][]byte)(nil), "cockroach.sql.jobs.jobspb.EncryptionInfo.EncryptedDataKeyByKMSMasterKeyIDEntry") + proto.RegisterType((*StreamIngestionDetails)(nil), "cockroach.sql.jobs.jobspb.StreamIngestionDetails") proto.RegisterType((*BackupDetails)(nil), "cockroach.sql.jobs.jobspb.BackupDetails") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.jobs.jobspb.BackupDetails.UrisByLocalityKvEntry") proto.RegisterType((*BackupProgress)(nil), "cockroach.sql.jobs.jobspb.BackupProgress") @@ -2723,6 +2763,30 @@ func (m *EncryptionInfo) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StreamIngestionDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.StreamAddress) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(len(m.StreamAddress))) + i += copy(dAtA[i:], m.StreamAddress) + } + return i, nil +} + func (m *BackupDetails) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4937,6 +5001,19 @@ func (m *EncryptionInfo) Size() (n int) { return n } +func (m *StreamIngestionDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.StreamAddress) + if l > 0 { + n += 1 + l + sovJobs(uint64(l)) + } + return n +} + func (m *BackupDetails) Size() (n int) { if m == nil { return 0 @@ -6572,6 +6649,85 @@ func (m *EncryptionInfo) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StreamIngestionDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StreamAddress = github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.StreamAddress(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *BackupDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -13066,282 +13222,287 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_98971d9474e7d77d) } - -var fileDescriptor_jobs_98971d9474e7d77d = []byte{ - // 4369 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x4b, 0x6c, 0x23, 0x47, - 0x7a, 0x16, 0x1f, 0x22, 0x9b, 0x3f, 0x45, 0xb2, 0x59, 0xd2, 0xcc, 0x70, 0x19, 0x5b, 0x54, 0xe8, - 0xd7, 0xcc, 0xd8, 0xa6, 0xbc, 0x72, 0xd6, 0x6b, 0x4f, 0xec, 0xf1, 0x8a, 0x0f, 0x49, 0xa4, 0x46, - 0x0f, 0x37, 0xa5, 0xf1, 0xda, 0x0b, 0xa7, 0xd3, 0xec, 0x2e, 0x49, 0x1d, 0x91, 0xdd, 0x3d, 0x5d, - 0xcd, 0x99, 0xd1, 0x26, 0x48, 0x82, 0x0d, 0x12, 0x2c, 0xe6, 0x94, 0x00, 0x9b, 0x1c, 0x92, 0x0c, - 0x10, 0x20, 0x59, 0x20, 0x87, 0x00, 0x01, 0x8c, 0x20, 0xc9, 0x21, 0xb7, 0x5c, 0x7c, 0x48, 0x80, - 0xbd, 0x04, 0x58, 0xe4, 0xc0, 0x4d, 0xe4, 0x4b, 0x2e, 0x01, 0x16, 0xc9, 0x6d, 0x4e, 0x41, 0x3d, - 0xba, 0xd9, 0xa4, 0x5e, 0xd4, 0xc8, 0xde, 0x5c, 0x24, 0xd6, 0x5f, 0x7f, 0x7d, 0x55, 0xf5, 0xd7, - 0xff, 0xac, 0x6a, 0xb8, 0xfe, 0x1b, 0x76, 0x87, 0x2c, 0xd2, 0x3f, 0x4e, 0x87, 0xfd, 0xab, 0x38, - 0xae, 0xed, 0xd9, 0xe8, 0x1b, 0xba, 0xad, 0x1f, 0xba, 0xb6, 0xa6, 0x1f, 0x54, 0xc8, 0x83, 0x6e, - 0x85, 0xf5, 0x70, 0xae, 0xe2, 0x35, 0xec, 0xba, 0xb6, 0x4b, 0xf9, 0xf9, 0x0f, 0x3e, 0xa2, 0x38, - 0xb7, 0x6f, 0xef, 0xdb, 0xec, 0xe7, 0x22, 0xfd, 0x25, 0xa8, 0x88, 0x61, 0x38, 0x9d, 0x45, 0x43, - 0xf3, 0x34, 0x41, 0x2b, 0xf8, 0x34, 0xd3, 0x7e, 0x73, 0xcf, 0x76, 0x7b, 0x9a, 0xe7, 0x63, 0xbc, - 0x44, 0x1e, 0x74, 0x17, 0x75, 0xcd, 0xd3, 0xba, 0xf6, 0xfe, 0xa2, 0x81, 0x89, 0xee, 0x74, 0x16, - 0x89, 0xe7, 0xf6, 0x75, 0xaf, 0xef, 0x62, 0x43, 0x30, 0x95, 0x4e, 0x61, 0xf2, 0xb0, 0xa5, 0x59, - 0x9e, 0x8f, 0xdf, 0xf7, 0xcc, 0xee, 0xe2, 0x41, 0x57, 0x5f, 0xf4, 0xcc, 0x1e, 0x26, 0x9e, 0xd6, - 0x73, 0x78, 0x4f, 0xf9, 0x77, 0x60, 0xfa, 0x1e, 0xd6, 0x08, 0x46, 0x9f, 0x42, 0xd2, 0xb2, 0x0d, - 0xac, 0x9a, 0x46, 0x21, 0xb2, 0x10, 0xb9, 0x99, 0xa9, 0x2e, 0x1f, 0x0f, 0x4a, 0x89, 0x4d, 0xdb, - 0xc0, 0xcd, 0xfa, 0xb3, 0x41, 0xe9, 0xed, 0x7d, 0xd3, 0x3b, 0xe8, 0x77, 0x2a, 0xba, 0xdd, 0x5b, - 0x0c, 0x04, 0x61, 0x74, 0x86, 0xbf, 0x17, 0x9d, 0xc3, 0xfd, 0x45, 0xb1, 0x8d, 0x0a, 0x1f, 0xa6, - 0x24, 0x28, 0x62, 0xd3, 0x40, 0x73, 0x30, 0x8d, 0x1d, 0x5b, 0x3f, 0x28, 0x44, 0x17, 0x22, 0x37, - 0x63, 0x0a, 0x6f, 0xdc, 0x89, 0xff, 0xd7, 0x5f, 0x94, 0x22, 0xe5, 0x1f, 0x47, 0xe1, 0x46, 0x55, - 0xd3, 0x0f, 0xfb, 0x4e, 0xc3, 0xd2, 0xdd, 0x23, 0xc7, 0x33, 0x6d, 0x6b, 0x8b, 0xfd, 0x25, 0x48, - 0x86, 0xd8, 0x21, 0x3e, 0x62, 0xeb, 0x99, 0x51, 0xe8, 0x4f, 0xf4, 0x01, 0xc4, 0x7b, 0xb6, 0x81, - 0x19, 0x50, 0x76, 0xe9, 0x56, 0xe5, 0xcc, 0x33, 0xa9, 0x0c, 0xd1, 0x36, 0x6c, 0x03, 0x2b, 0x6c, - 0x18, 0xea, 0x80, 0x74, 0xd8, 0x23, 0xaa, 0x69, 0xed, 0xd9, 0x85, 0xd8, 0x42, 0xe4, 0x66, 0x7a, - 0xe9, 0xce, 0x39, 0x10, 0x67, 0x2c, 0xab, 0xb2, 0xbe, 0xd1, 0x6e, 0x5a, 0x7b, 0x76, 0x35, 0x7d, - 0x3c, 0x28, 0x25, 0x45, 0x43, 0x49, 0x1e, 0xf6, 0x08, 0xfd, 0x51, 0xdc, 0x02, 0x9f, 0x46, 0xd7, - 0xdf, 0x77, 0x4d, 0xb6, 0xfe, 0x94, 0x42, 0x7f, 0xa2, 0x37, 0x00, 0x61, 0x8e, 0x87, 0x0d, 0x95, - 0x2a, 0x80, 0x4a, 0x37, 0x18, 0x65, 0x1b, 0x94, 0x83, 0x9e, 0xba, 0xe6, 0x69, 0xeb, 0xf8, 0x88, - 0x4b, 0x48, 0xc8, 0xe9, 0x77, 0x63, 0x90, 0x1d, 0x2e, 0x85, 0xc1, 0xaf, 0x41, 0x82, 0xe8, 0x07, - 0xb8, 0x87, 0xd9, 0x0c, 0xd9, 0xa5, 0xb7, 0x26, 0x12, 0x07, 0x1d, 0x5a, 0x69, 0xb3, 0x71, 0x8a, - 0x18, 0x8f, 0x10, 0xc4, 0x89, 0xd6, 0xf5, 0xc4, 0x42, 0xd8, 0x6f, 0xf4, 0x67, 0x11, 0x58, 0x18, - 0x5f, 0x51, 0xf5, 0x68, 0x7d, 0xa3, 0xbd, 0xa1, 0x11, 0x0f, 0xbb, 0xeb, 0xf8, 0xa8, 0x59, 0x2f, - 0xc4, 0x16, 0x62, 0x37, 0xd3, 0x4b, 0x5b, 0x93, 0x4f, 0xdc, 0xb8, 0x00, 0xb1, 0x61, 0x79, 0xee, - 0x91, 0x72, 0xe1, 0xc4, 0xc5, 0x36, 0xbc, 0x32, 0x11, 0x54, 0x58, 0x87, 0x52, 0x5c, 0x87, 0xe6, - 0x60, 0xfa, 0xa1, 0xd6, 0xed, 0x63, 0xb1, 0x5b, 0xde, 0xb8, 0x13, 0x7d, 0x37, 0x52, 0xbe, 0x01, - 0x09, 0x2e, 0x18, 0x94, 0x81, 0xd4, 0x72, 0xa3, 0xbd, 0xf4, 0xad, 0x77, 0x56, 0x6b, 0x1b, 0xf2, - 0x94, 0x38, 0x82, 0x7f, 0x4a, 0x40, 0x86, 0xeb, 0x44, 0x1d, 0x7b, 0x9a, 0xd9, 0x25, 0xa8, 0x0a, - 0x40, 0x3c, 0xcd, 0xf5, 0x54, 0x6a, 0x56, 0x6c, 0x8e, 0xf4, 0xd2, 0x8b, 0x21, 0x61, 0x50, 0xb3, - 0xab, 0x1c, 0x74, 0xf5, 0xca, 0x8e, 0x6f, 0x76, 0xd5, 0xf8, 0x17, 0x83, 0xd2, 0x94, 0x92, 0x62, - 0xc3, 0x28, 0x15, 0xdd, 0x05, 0x09, 0x5b, 0x06, 0x47, 0x88, 0x4e, 0x8e, 0x90, 0xc4, 0x96, 0xc1, - 0xc6, 0x7f, 0x83, 0x2b, 0x19, 0x55, 0xe7, 0x54, 0x35, 0x79, 0x3c, 0x28, 0xc5, 0x76, 0x95, 0x26, - 0xd7, 0xb6, 0xf7, 0xa1, 0x68, 0x60, 0xc7, 0xc5, 0xba, 0x46, 0xd5, 0xad, 0xc3, 0x96, 0xae, 0xf6, - 0x34, 0xcb, 0xdc, 0xc3, 0xc4, 0x2b, 0xc4, 0xd9, 0xf6, 0x0b, 0x43, 0x0e, 0xbe, 0xb7, 0x0d, 0xd1, - 0x8f, 0x7e, 0x2f, 0x02, 0xb3, 0x7d, 0xd7, 0x24, 0x6a, 0xe7, 0x48, 0xed, 0xda, 0xba, 0xd6, 0x35, - 0xbd, 0x23, 0xf5, 0xf0, 0x61, 0x61, 0x9a, 0x9d, 0xf9, 0xdd, 0x0b, 0x0d, 0x47, 0x08, 0xa9, 0xb2, - 0xeb, 0x9a, 0xa4, 0x7a, 0x74, 0x4f, 0x20, 0xac, 0x3f, 0x64, 0xe7, 0x52, 0x9d, 0x3b, 0x1e, 0x94, - 0xe4, 0x5d, 0xa5, 0x19, 0xee, 0xba, 0xaf, 0xc8, 0xfd, 0x31, 0x66, 0xa4, 0x05, 0x16, 0x63, 0xda, - 0x96, 0x6a, 0x73, 0x13, 0x2c, 0x24, 0x98, 0xa0, 0x96, 0x2e, 0x6f, 0xbc, 0x4a, 0x1e, 0x9f, 0x70, - 0x33, 0x7f, 0x14, 0x81, 0x22, 0xf5, 0x86, 0x58, 0xa7, 0x62, 0x0a, 0x3c, 0xa4, 0xea, 0x62, 0xdd, - 0x76, 0x8d, 0x42, 0x92, 0xca, 0xa9, 0xda, 0xfe, 0xf7, 0x49, 0x9d, 0x20, 0xf3, 0xb5, 0xfd, 0xbe, - 0x69, 0x54, 0x76, 0x77, 0x9b, 0xf5, 0xe3, 0x41, 0xa9, 0xb0, 0xed, 0x83, 0x07, 0x87, 0xa8, 0x30, - 0x68, 0xa5, 0xe0, 0x9c, 0xd1, 0x83, 0xde, 0x85, 0xac, 0x6e, 0x77, 0xbb, 0x58, 0x67, 0xdb, 0xde, - 0x55, 0x9a, 0x05, 0x89, 0x1d, 0x70, 0xfe, 0x78, 0x50, 0xca, 0xd4, 0x82, 0x1e, 0x7a, 0xd4, 0x19, - 0x3d, 0xdc, 0x44, 0x0a, 0xe4, 0x42, 0x02, 0x63, 0xae, 0x2e, 0xc5, 0xa4, 0x75, 0x6b, 0x62, 0x2b, - 0x55, 0xb2, 0x78, 0xa4, 0x5d, 0xac, 0xc1, 0xb5, 0x53, 0x4f, 0xf1, 0x22, 0xeb, 0x4a, 0x85, 0xad, - 0x4b, 0x86, 0x2c, 0x3f, 0x94, 0x6d, 0xd7, 0xde, 0x77, 0x31, 0x21, 0xe5, 0xcf, 0xb3, 0x90, 0x55, - 0x30, 0xf1, 0x6c, 0x17, 0xfb, 0x16, 0xf5, 0x79, 0x04, 0x66, 0x69, 0x04, 0x73, 0x4d, 0xc7, 0xb3, - 0x5d, 0xd5, 0xc5, 0x8f, 0x5c, 0xd3, 0xc3, 0xa4, 0x10, 0x65, 0x4a, 0xb7, 0x7c, 0xce, 0x16, 0x46, - 0x81, 0x2a, 0xf5, 0x00, 0x44, 0x11, 0x18, 0x5c, 0xef, 0xee, 0xfe, 0xe0, 0x67, 0xa5, 0x3b, 0x13, - 0x9d, 0xe3, 0xc9, 0xa0, 0x5a, 0x69, 0xd6, 0x15, 0x64, 0x9c, 0x00, 0x46, 0x2f, 0x40, 0x9c, 0xea, - 0x2d, 0xf3, 0x86, 0xa9, 0xaa, 0x74, 0x3c, 0x28, 0xc5, 0xa9, 0x66, 0x2b, 0x8c, 0x3a, 0x62, 0xe0, - 0xf1, 0xe7, 0x30, 0xf0, 0x55, 0x48, 0x7b, 0x5a, 0xa7, 0x8b, 0x55, 0x3a, 0x33, 0x11, 0xe6, 0xf7, - 0xea, 0x98, 0x24, 0xc8, 0x83, 0x6e, 0x47, 0x23, 0xb8, 0xb2, 0x43, 0x39, 0x43, 0x7b, 0x07, 0xcf, - 0x27, 0x10, 0xb4, 0x08, 0x69, 0xfb, 0x21, 0x76, 0x5d, 0xd3, 0xc0, 0xaa, 0xd1, 0x61, 0x36, 0x94, - 0xaa, 0x66, 0x8f, 0x07, 0x25, 0xd8, 0x12, 0xe4, 0x7a, 0x55, 0x01, 0x9f, 0xa5, 0xde, 0x41, 0x1e, - 0xcc, 0x09, 0xa7, 0x11, 0xd8, 0x3f, 0xd3, 0xa7, 0x24, 0x5b, 0xc2, 0xfb, 0x93, 0x1f, 0x06, 0x3f, - 0x77, 0x5f, 0x79, 0x58, 0xf0, 0xe4, 0x9b, 0x44, 0x9d, 0x13, 0x3d, 0xe8, 0x75, 0xc8, 0x3b, 0x2e, - 0x76, 0x34, 0x17, 0xab, 0xba, 0xdd, 0x73, 0xba, 0xd8, 0xc3, 0x06, 0xd3, 0x7e, 0x49, 0x91, 0x45, - 0x47, 0xcd, 0xa7, 0xa3, 0x57, 0x20, 0x4b, 0x3c, 0xcd, 0xa3, 0x31, 0x9d, 0x60, 0x97, 0x72, 0xa6, - 0x18, 0x67, 0x86, 0x51, 0x9b, 0x82, 0x88, 0xde, 0x86, 0x6b, 0xc3, 0x73, 0x23, 0xaa, 0xd3, 0xef, - 0x74, 0x4d, 0x72, 0x80, 0x8d, 0x02, 0x30, 0xee, 0xb9, 0x50, 0xe7, 0xb6, 0xdf, 0x87, 0x8e, 0x46, - 0x54, 0x51, 0xa7, 0x82, 0xd1, 0xf6, 0x71, 0x21, 0xbd, 0x10, 0xb9, 0x39, 0x5d, 0x5d, 0x7b, 0x36, - 0x28, 0xd5, 0x27, 0xd6, 0x23, 0x82, 0x7b, 0x8b, 0x9e, 0x8b, 0x71, 0x48, 0x2d, 0x6b, 0x02, 0x2f, - 0xac, 0x51, 0x3e, 0x0d, 0x29, 0x00, 0x43, 0x13, 0x2c, 0xcc, 0x3c, 0xb7, 0xb7, 0x0b, 0xa1, 0xa0, - 0x65, 0x48, 0xf2, 0xa4, 0x90, 0x14, 0x32, 0xec, 0x00, 0x7f, 0xf9, 0x2c, 0x1d, 0x62, 0x5c, 0xa1, - 0x53, 0xf2, 0xc7, 0xa1, 0x3a, 0x80, 0x77, 0xe4, 0xf8, 0x9a, 0x98, 0x65, 0x28, 0xaf, 0x9c, 0x85, - 0x72, 0xe4, 0x84, 0x15, 0x31, 0xe5, 0x89, 0x36, 0x41, 0x2d, 0x98, 0x61, 0x79, 0x87, 0x26, 0x70, - 0x72, 0x0c, 0xe7, 0xb5, 0x33, 0x70, 0x58, 0x44, 0xd6, 0x42, 0x48, 0x69, 0x12, 0x50, 0x08, 0xda, - 0x86, 0x2c, 0x4d, 0xa3, 0x28, 0xa7, 0x40, 0x93, 0x19, 0xda, 0xad, 0x33, 0xd0, 0xea, 0x82, 0x39, - 0x84, 0x97, 0x31, 0x42, 0x34, 0x52, 0xfc, 0xdf, 0x08, 0xe4, 0x4f, 0x38, 0x0f, 0xb4, 0x03, 0xd1, - 0x20, 0x33, 0xa6, 0x3e, 0x3d, 0xca, 0xb2, 0xe2, 0xab, 0x38, 0x92, 0xa8, 0x69, 0xa0, 0x7d, 0x48, - 0x51, 0x75, 0xb6, 0x3c, 0x9a, 0x76, 0x47, 0x19, 0x78, 0xeb, 0x78, 0x50, 0x92, 0xb6, 0x19, 0xf1, - 0xca, 0x53, 0x48, 0x1c, 0xbc, 0x69, 0xa0, 0x12, 0xa4, 0x3d, 0x5b, 0xc5, 0x8f, 0x4d, 0xe2, 0x99, - 0xd6, 0x3e, 0x4b, 0x16, 0x24, 0x05, 0x3c, 0xbb, 0x21, 0x28, 0xc5, 0x3f, 0x8f, 0x02, 0x3a, 0x69, - 0xa5, 0xe8, 0x1f, 0x23, 0xf0, 0x82, 0x9f, 0x03, 0xd8, 0xae, 0xb9, 0x6f, 0x5a, 0x5a, 0x77, 0x24, - 0x19, 0x88, 0x30, 0x69, 0x7f, 0x7a, 0x15, 0x57, 0x20, 0x12, 0x84, 0x2d, 0x01, 0x3f, 0x9e, 0x28, - 0xbc, 0x40, 0x23, 0x28, 0x4f, 0x14, 0x4e, 0xb0, 0xdc, 0x57, 0x0a, 0xfd, 0x33, 0x06, 0x17, 0xd7, - 0xe1, 0xc5, 0x73, 0x81, 0x2f, 0x13, 0xbb, 0x8a, 0x3f, 0x88, 0xc0, 0x8d, 0x33, 0x22, 0x4a, 0x18, - 0x27, 0xc3, 0x71, 0x3e, 0x0a, 0xe3, 0xa4, 0x97, 0x7e, 0xf5, 0x0a, 0x51, 0x2b, 0xb4, 0x88, 0x56, - 0x5c, 0x8a, 0xc8, 0xd1, 0xf2, 0x5b, 0x90, 0x13, 0x83, 0xfc, 0x38, 0x8a, 0x5e, 0x04, 0x38, 0x30, - 0xf7, 0x0f, 0xd4, 0x47, 0x9a, 0x87, 0x5d, 0x51, 0x2e, 0xa5, 0x28, 0xe5, 0x63, 0x4a, 0x28, 0xff, - 0x9b, 0x04, 0x99, 0x66, 0xcf, 0xb1, 0x5d, 0xcf, 0x8f, 0xb2, 0xf7, 0x20, 0xc1, 0xe2, 0x02, 0x11, - 0xe7, 0x57, 0x39, 0x67, 0x85, 0x23, 0x23, 0x79, 0x7c, 0x11, 0x6e, 0x41, 0x60, 0x04, 0xe1, 0x2f, - 0x7a, 0x6a, 0xf8, 0xfb, 0x00, 0x12, 0xbc, 0xa4, 0x15, 0x15, 0x57, 0x29, 0x34, 0x97, 0x5f, 0x2f, - 0x36, 0xb7, 0x56, 0xcc, 0x2e, 0x5e, 0x61, 0x6c, 0x3e, 0x38, 0x1f, 0x84, 0x5e, 0x05, 0x89, 0x10, - 0x4f, 0x25, 0xe6, 0xf7, 0x79, 0xf4, 0x8c, 0xf1, 0xb2, 0xab, 0xdd, 0xde, 0x69, 0x9b, 0xdf, 0xc7, - 0x4a, 0x92, 0x10, 0x8f, 0xfe, 0x40, 0x45, 0x90, 0x1e, 0x69, 0xdd, 0x2e, 0x8b, 0xb2, 0xd3, 0xac, - 0xcc, 0x0c, 0xda, 0xa3, 0x66, 0x96, 0xf8, 0x7a, 0xcd, 0x4c, 0x04, 0x4c, 0x47, 0xf3, 0x0e, 0x58, - 0xe6, 0x98, 0x52, 0x80, 0x93, 0xb6, 0x35, 0xef, 0x00, 0x15, 0x20, 0x49, 0x34, 0x1a, 0xbb, 0x48, - 0x41, 0x5a, 0x88, 0xdd, 0x9c, 0x51, 0xfc, 0x26, 0x9a, 0x07, 0x16, 0x79, 0x79, 0x93, 0x05, 0xb1, - 0x98, 0x12, 0xa2, 0x30, 0x39, 0x1c, 0x9a, 0x8e, 0xba, 0x77, 0x48, 0x78, 0xd0, 0x12, 0x72, 0x38, - 0x34, 0x9d, 0x95, 0x75, 0xa2, 0x24, 0x69, 0xe7, 0xca, 0x21, 0x41, 0xaf, 0x41, 0xce, 0xb4, 0xf6, - 0x31, 0xf1, 0x54, 0xc3, 0x74, 0xb1, 0xee, 0x75, 0x8f, 0x58, 0xc0, 0x92, 0x94, 0x2c, 0x27, 0xd7, - 0x05, 0x15, 0xdd, 0x02, 0x79, 0x3c, 0xcc, 0xb2, 0x40, 0x23, 0x29, 0xb9, 0xb1, 0x28, 0x4b, 0x59, - 0xf9, 0x51, 0x87, 0x02, 0x67, 0x86, 0xb3, 0x72, 0xfa, 0x30, 0x66, 0x56, 0x60, 0xd6, 0xd1, 0x5c, - 0x82, 0xd5, 0x4e, 0xdf, 0x32, 0xba, 0x58, 0xe5, 0xbe, 0xba, 0x90, 0x65, 0xdc, 0x79, 0xd6, 0x55, - 0x65, 0x3d, 0xdc, 0xad, 0x5f, 0x94, 0x7b, 0x5f, 0xff, 0x7f, 0xc8, 0xbd, 0x8b, 0x3f, 0x8e, 0xc2, - 0x34, 0xd3, 0x73, 0x74, 0x07, 0xe2, 0xf4, 0x98, 0x45, 0x65, 0x37, 0x69, 0xce, 0xc5, 0xc6, 0xd0, - 0x9a, 0xda, 0xd2, 0x7a, 0xb8, 0x80, 0x98, 0x12, 0xb0, 0xdf, 0xe8, 0x06, 0x24, 0x09, 0x7e, 0xa0, - 0x3e, 0xd4, 0xba, 0x85, 0x59, 0x76, 0xc2, 0x09, 0x82, 0x1f, 0xdc, 0xd7, 0xba, 0xe8, 0x1a, 0x24, - 0x4c, 0xa2, 0x5a, 0xf8, 0x51, 0x61, 0x8e, 0x49, 0x6a, 0xda, 0x24, 0x9b, 0xf8, 0x11, 0x73, 0xdb, - 0x9a, 0xbb, 0x8f, 0x3d, 0x55, 0xb7, 0xbb, 0xa4, 0x70, 0x8d, 0x1a, 0x18, 0x4d, 0xe9, 0x28, 0xa9, - 0x66, 0x77, 0x09, 0xfa, 0x25, 0x48, 0x3d, 0xd2, 0x88, 0x8a, 0x7b, 0x8e, 0x77, 0xc4, 0x84, 0x25, - 0x51, 0xb5, 0x27, 0x0d, 0xda, 0x6e, 0xc5, 0xa5, 0xa8, 0x1c, 0x6b, 0xc5, 0xa5, 0x98, 0x1c, 0x6f, - 0xc5, 0xa5, 0xb8, 0x3c, 0xdd, 0x8a, 0x4b, 0xd3, 0x72, 0xa2, 0x15, 0x97, 0x12, 0x72, 0xb2, 0x15, - 0x97, 0x92, 0xb2, 0xd4, 0x8a, 0x4b, 0x92, 0x9c, 0x6a, 0xc5, 0xa5, 0x94, 0x0c, 0xad, 0xb8, 0x04, - 0x72, 0xba, 0x15, 0x97, 0xd2, 0xf2, 0x4c, 0x2b, 0x2e, 0xcd, 0xc8, 0x99, 0x56, 0x5c, 0xca, 0xc8, - 0xd9, 0x56, 0x5c, 0xca, 0xca, 0xb9, 0x56, 0x5c, 0xca, 0xc9, 0x72, 0x2b, 0x2e, 0xc9, 0x72, 0xbe, - 0x15, 0x97, 0xf2, 0x32, 0x2a, 0x7f, 0x1e, 0x01, 0xb9, 0x8d, 0x1f, 0xf4, 0xb1, 0xa5, 0xe3, 0xfb, - 0x5a, 0xb7, 0x76, 0xd0, 0xb7, 0x0e, 0xd1, 0xab, 0x90, 0xd3, 0xe9, 0x0f, 0x95, 0x17, 0xc6, 0x74, - 0xab, 0x11, 0xb6, 0xd5, 0x0c, 0x23, 0xb7, 0x29, 0x95, 0xee, 0xf8, 0x45, 0x00, 0xc1, 0x47, 0x2d, - 0x9b, 0x5f, 0x0c, 0xa5, 0x38, 0x0b, 0x35, 0xe7, 0x31, 0x18, 0xd7, 0x7e, 0xc4, 0xdc, 0xc7, 0x08, - 0x8c, 0x62, 0x3f, 0x42, 0x8b, 0x30, 0x67, 0xe1, 0xc7, 0x9e, 0x3a, 0xce, 0xcc, 0x5c, 0x85, 0x92, - 0xa7, 0x7d, 0xb5, 0xf0, 0x80, 0xf2, 0xbf, 0x46, 0x21, 0xe7, 0x2f, 0xda, 0x77, 0x87, 0x7b, 0x20, - 0xd3, 0x63, 0x31, 0x0d, 0xd5, 0xb3, 0x39, 0x92, 0xef, 0x18, 0x3f, 0x38, 0xc7, 0x31, 0x8e, 0xa1, - 0xd0, 0x76, 0xd3, 0xd8, 0xb1, 0xd9, 0x74, 0x3c, 0x34, 0x28, 0x19, 0x12, 0xa6, 0x15, 0x77, 0x21, - 0xeb, 0x0f, 0xe2, 0x14, 0x54, 0x83, 0xc4, 0xc8, 0x7c, 0xaf, 0x4f, 0x30, 0x9f, 0x2f, 0x6a, 0x45, - 0x0c, 0x2d, 0xfe, 0x26, 0xa0, 0x93, 0x73, 0x87, 0xc3, 0xd2, 0x34, 0x0f, 0x4b, 0x5b, 0xa3, 0x61, - 0xe9, 0xbd, 0xcb, 0xed, 0x2d, 0xb4, 0xec, 0x70, 0x55, 0xf7, 0xcf, 0x51, 0xc8, 0xf2, 0x10, 0x11, - 0x84, 0xa3, 0xd7, 0x21, 0xcf, 0x9c, 0x96, 0x69, 0xed, 0xab, 0x8e, 0x20, 0xb2, 0xfd, 0x45, 0x15, - 0xd9, 0xef, 0x08, 0x98, 0x5f, 0x82, 0x8c, 0x8b, 0x35, 0x63, 0xc8, 0x18, 0x65, 0x8c, 0x33, 0x94, - 0x18, 0x30, 0xbd, 0x02, 0x59, 0x16, 0x0d, 0x87, 0x5c, 0x31, 0xc6, 0x95, 0x61, 0xd4, 0x80, 0xad, - 0x0a, 0x19, 0xe2, 0x68, 0xd6, 0x90, 0x2b, 0xce, 0x84, 0x7a, 0xe3, 0x94, 0x88, 0xd3, 0x76, 0x34, - 0x4b, 0x44, 0x9a, 0x19, 0x3a, 0x26, 0x1c, 0x4b, 0x5d, 0x4c, 0xfa, 0x3d, 0xac, 0x3a, 0x36, 0x2f, - 0xb6, 0x62, 0x4a, 0x8a, 0x53, 0xb6, 0x6d, 0x82, 0x76, 0x99, 0xaa, 0x30, 0x59, 0xa8, 0x06, 0x17, - 0x4e, 0x21, 0xc1, 0x66, 0xb9, 0x3d, 0xb9, 0x38, 0x95, 0x1c, 0x19, 0x25, 0x94, 0x7f, 0x0b, 0x6e, - 0xd0, 0x7c, 0x99, 0x3b, 0xc5, 0xda, 0x81, 0x66, 0xed, 0x07, 0xca, 0xa9, 0x41, 0x92, 0xe5, 0xdc, - 0x41, 0xfa, 0xb9, 0x76, 0x3c, 0x28, 0x25, 0x28, 0xf7, 0x95, 0x03, 0x57, 0x82, 0x02, 0x37, 0x8d, - 0x72, 0x11, 0x0a, 0xe3, 0xb3, 0x07, 0x35, 0xba, 0xc2, 0x4a, 0xf4, 0x7e, 0x0f, 0x53, 0x89, 0xdd, - 0x33, 0x89, 0x87, 0xbe, 0x03, 0x33, 0x42, 0x42, 0x54, 0x70, 0xbe, 0xe6, 0x5e, 0x20, 0xe4, 0xb4, - 0x1b, 0x80, 0x90, 0xf2, 0xdf, 0x45, 0x60, 0xb6, 0xee, 0xda, 0x8e, 0x83, 0x0d, 0xe1, 0x3b, 0xf9, - 0x56, 0x7d, 0x97, 0x19, 0x09, 0xb9, 0xcc, 0x4d, 0x88, 0x36, 0xeb, 0x22, 0x37, 0xbe, 0x7b, 0xd5, - 0x94, 0xbb, 0x59, 0x47, 0xef, 0x41, 0x82, 0x96, 0x86, 0x7d, 0xc2, 0xfc, 0x49, 0xf6, 0x44, 0x11, - 0x34, 0x72, 0x6c, 0x8c, 0x51, 0x11, 0x03, 0xca, 0x7f, 0x90, 0x80, 0x6b, 0x61, 0x19, 0xad, 0xd6, - 0xfc, 0x85, 0x7f, 0x06, 0x49, 0xd3, 0x32, 0xf0, 0x63, 0x3c, 0x91, 0xdf, 0x38, 0x0d, 0xa2, 0x22, - 0xe4, 0xd1, 0xa4, 0x30, 0x7e, 0xd9, 0x25, 0x30, 0xd1, 0x77, 0x83, 0x74, 0x8d, 0x5f, 0x83, 0xdc, - 0x79, 0x6e, 0xf4, 0xfa, 0x58, 0xea, 0x36, 0x92, 0x19, 0x31, 0x07, 0xfb, 0x35, 0x65, 0x46, 0x6d, - 0xc8, 0x9b, 0x96, 0x87, 0xdd, 0x2e, 0xd6, 0x1e, 0xd2, 0x40, 0x4f, 0xa7, 0x17, 0xb7, 0x21, 0x93, - 0x86, 0x55, 0x39, 0x04, 0xc0, 0xc3, 0xf3, 0x67, 0x30, 0x1b, 0x06, 0xf5, 0x8f, 0xe0, 0xfc, 0x1b, - 0x12, 0x26, 0xe1, 0x21, 0xac, 0x7f, 0x11, 0x11, 0x02, 0x6a, 0x72, 0x9c, 0xe2, 0x9f, 0x44, 0x60, - 0x26, 0x7c, 0x2c, 0xc8, 0x04, 0x89, 0xcd, 0xe1, 0xdb, 0x62, 0xac, 0xba, 0x49, 0x73, 0x30, 0xd6, - 0xc9, 0x64, 0xf5, 0xe1, 0x73, 0xcb, 0x8a, 0x43, 0x88, 0x23, 0x6f, 0x1a, 0x34, 0xb0, 0x1b, 0xae, - 0xed, 0x0c, 0xaf, 0x85, 0x63, 0x8a, 0x44, 0x09, 0x34, 0x57, 0x29, 0xfe, 0x36, 0xa4, 0x82, 0x03, - 0x0d, 0x55, 0xa6, 0xb1, 0xaf, 0xb0, 0x32, 0x3d, 0x6f, 0xfe, 0xf2, 0xcf, 0x12, 0x30, 0x7b, 0x9a, - 0xab, 0xfa, 0x04, 0xe4, 0x90, 0x67, 0x50, 0xbb, 0x26, 0xf1, 0x84, 0xc6, 0xde, 0x3a, 0xbf, 0x04, - 0x0a, 0xb9, 0x17, 0x71, 0x1e, 0x59, 0x77, 0xd4, 0xe9, 0x7c, 0x0f, 0xb2, 0x06, 0xdf, 0xb2, 0x2a, - 0x4c, 0x21, 0x76, 0x61, 0xe5, 0x72, 0x8a, 0x8b, 0x11, 0xe8, 0x19, 0x23, 0xd4, 0x45, 0xd8, 0x4d, - 0xb7, 0x8f, 0x1e, 0xdc, 0x26, 0x98, 0x06, 0xd3, 0xcf, 0x4c, 0xb5, 0x7d, 0x3c, 0x28, 0xe5, 0x05, - 0x96, 0x7f, 0x7d, 0x70, 0x65, 0x19, 0xe7, 0x8d, 0x31, 0x40, 0x83, 0x3a, 0x7a, 0xda, 0x4f, 0x27, - 0x9e, 0x1e, 0x3a, 0x7a, 0xaa, 0xa9, 0x57, 0x77, 0xf4, 0xf4, 0x67, 0xd3, 0x40, 0xbf, 0x1f, 0x81, - 0x3c, 0xbf, 0x4b, 0xec, 0xf5, 0x3d, 0x8d, 0x5f, 0x10, 0xfb, 0x15, 0xd1, 0x27, 0xc7, 0x83, 0x52, - 0x8e, 0x09, 0x64, 0x43, 0xf4, 0xb1, 0x69, 0xab, 0xcf, 0x3b, 0xed, 0x10, 0x45, 0x54, 0x09, 0x01, - 0xc1, 0x40, 0xeb, 0x90, 0xe5, 0xe5, 0x9d, 0x4a, 0x0b, 0x1c, 0xd3, 0xb6, 0x58, 0xa9, 0x94, 0xa9, - 0xbe, 0xfc, 0x6c, 0x50, 0x5a, 0x38, 0x45, 0xb3, 0x78, 0x65, 0x78, 0x9f, 0xf3, 0x2a, 0x99, 0xbd, - 0x70, 0x13, 0xe9, 0x90, 0x09, 0x54, 0xe3, 0xc8, 0x11, 0x95, 0xd5, 0xd5, 0x83, 0xc5, 0x8c, 0xaf, - 0x23, 0x14, 0x13, 0xed, 0x43, 0xce, 0x9f, 0x84, 0x97, 0x34, 0xa4, 0x90, 0xfa, 0x4a, 0xa6, 0xf1, - 0xd5, 0x9a, 0xef, 0x9a, 0x88, 0x22, 0xff, 0x3a, 0xcc, 0x9d, 0x1a, 0x8d, 0xff, 0x74, 0x1a, 0xae, - 0x8f, 0x7a, 0xf8, 0x20, 0x71, 0x51, 0xc7, 0x63, 0xd0, 0x87, 0x13, 0x47, 0x09, 0x1f, 0x83, 0xbb, - 0x21, 0xbf, 0x35, 0x1e, 0x85, 0x3e, 0x1b, 0x8b, 0x42, 0xcf, 0x81, 0xcf, 0xd4, 0x6b, 0x0c, 0x5f, - 0x80, 0x16, 0xff, 0x25, 0x02, 0x99, 0x91, 0xf9, 0x7f, 0x91, 0xee, 0x76, 0x3b, 0xc8, 0x0a, 0xf8, - 0xcb, 0xf2, 0xbb, 0x97, 0xdf, 0xdb, 0x68, 0xb2, 0x50, 0xfc, 0x87, 0x08, 0x64, 0x46, 0xb6, 0xfb, - 0x35, 0x39, 0xea, 0xaf, 0x7c, 0xe5, 0xe5, 0x6f, 0x43, 0x82, 0x53, 0x10, 0x82, 0xec, 0xc7, 0xcb, - 0xcd, 0x9d, 0xe6, 0xe6, 0xaa, 0xba, 0xb2, 0xa5, 0xa8, 0xab, 0x35, 0x79, 0x0a, 0xcd, 0x80, 0x54, - 0x6f, 0xdc, 0x6b, 0x50, 0xa2, 0x1c, 0x41, 0x69, 0x48, 0xb2, 0x56, 0xa3, 0x2e, 0x47, 0xcb, 0x55, - 0x90, 0x39, 0xf6, 0x1e, 0xa6, 0xae, 0x95, 0x16, 0xa9, 0xa8, 0x02, 0xb3, 0x14, 0x16, 0xf7, 0x68, - 0x8e, 0x41, 0x83, 0x89, 0x1a, 0xca, 0xf0, 0xf2, 0x41, 0x17, 0x0d, 0x2b, 0x9b, 0x5a, 0x0f, 0x97, - 0xff, 0x3e, 0x0e, 0xf9, 0x21, 0x88, 0x1f, 0x58, 0x5e, 0x05, 0x89, 0x98, 0xd6, 0xa1, 0x3a, 0x7c, - 0xe8, 0xe4, 0x97, 0x1f, 0xa6, 0x75, 0xb8, 0xab, 0x34, 0x95, 0x24, 0xed, 0xdc, 0x75, 0x4d, 0xd4, - 0x82, 0xb8, 0xed, 0x78, 0x7e, 0xde, 0xff, 0xce, 0x39, 0xa2, 0x38, 0x31, 0x47, 0x65, 0xcb, 0xf1, - 0x44, 0xd5, 0xc6, 0x30, 0xd0, 0xdf, 0x44, 0x20, 0xc9, 0x2b, 0x6d, 0x3f, 0xc3, 0x7f, 0xef, 0x52, - 0x78, 0x5c, 0x00, 0xe2, 0xd5, 0xe9, 0x63, 0xaa, 0xea, 0xcf, 0x06, 0xa5, 0xfc, 0xb8, 0x80, 0xc8, - 0x15, 0x9f, 0xa3, 0xfc, 0x25, 0xa2, 0x16, 0x7f, 0x08, 0x19, 0x0a, 0x9a, 0xb9, 0xd4, 0x09, 0xdf, - 0x9a, 0x32, 0x23, 0x07, 0x51, 0xdc, 0x87, 0x99, 0xf0, 0xea, 0x4f, 0xb9, 0xe1, 0x5c, 0x1e, 0x2d, - 0x25, 0x5f, 0x9f, 0x48, 0x32, 0x1c, 0x33, 0x7c, 0xad, 0xfa, 0x6d, 0x48, 0x05, 0x62, 0xbf, 0xcc, - 0x7d, 0x2c, 0xf7, 0x92, 0xc1, 0x05, 0xc7, 0xb4, 0x9c, 0x28, 0xff, 0x6d, 0x04, 0x66, 0x14, 0x4c, - 0xec, 0xee, 0x43, 0x6c, 0xd0, 0xac, 0x01, 0x7d, 0x13, 0xe2, 0x34, 0x0b, 0x11, 0x77, 0x37, 0x17, - 0x94, 0x27, 0x8c, 0x15, 0x2d, 0x43, 0x2a, 0xb8, 0x81, 0xba, 0xcc, 0x5b, 0xfc, 0x70, 0x14, 0xba, - 0x05, 0x72, 0xc7, 0xee, 0x5b, 0x86, 0xe6, 0x1e, 0xa9, 0x2e, 0xd6, 0xf4, 0x03, 0x6c, 0x88, 0xdb, - 0xf6, 0x9c, 0x4f, 0x57, 0x38, 0xb9, 0xfc, 0xc3, 0x28, 0xa0, 0xa1, 0x70, 0x42, 0x6e, 0x82, 0xe6, - 0x3e, 0x6c, 0x1f, 0xa2, 0xc0, 0x8a, 0x9e, 0xfa, 0x3e, 0x32, 0x96, 0x42, 0x05, 0x1b, 0xf7, 0x8f, - 0xd4, 0x0d, 0xd1, 0x08, 0xfa, 0xe3, 0xf3, 0xef, 0xd9, 0x62, 0xec, 0x9e, 0x8d, 0x69, 0xe9, 0x2f, - 0xf4, 0xae, 0x4d, 0x84, 0xbb, 0xff, 0x89, 0x03, 0xaa, 0xb9, 0x58, 0xf3, 0x30, 0xf5, 0x3c, 0xe4, - 0xbc, 0x7a, 0xb0, 0x0a, 0xd3, 0xbc, 0x78, 0x88, 0x5e, 0xa6, 0x78, 0x10, 0x42, 0xe1, 0x43, 0xd1, - 0xaf, 0xc1, 0x8c, 0x6e, 0x77, 0xfb, 0x3d, 0x4b, 0x65, 0xaf, 0x84, 0x22, 0x95, 0xfc, 0xd6, 0x79, - 0x4a, 0x7c, 0x62, 0x71, 0x95, 0x9a, 0xdd, 0xa5, 0x6d, 0xbf, 0xbe, 0xe5, 0x80, 0x8c, 0x03, 0xbd, - 0x00, 0xa9, 0xc0, 0xa0, 0x58, 0x12, 0x99, 0x52, 0x86, 0x04, 0xb4, 0x04, 0xd3, 0x1a, 0x51, 0xed, - 0x3d, 0x96, 0xe5, 0x5d, 0xa4, 0x61, 0x4a, 0x5c, 0x23, 0x5b, 0x7b, 0xe8, 0x6d, 0xc8, 0xec, 0x3d, - 0xe0, 0x99, 0x2f, 0x77, 0xa0, 0xfc, 0xf1, 0x36, 0x77, 0x3c, 0x28, 0xa5, 0x57, 0x3e, 0x62, 0x9b, - 0xa5, 0xee, 0x53, 0x49, 0xef, 0x3d, 0x08, 0x1a, 0xe8, 0x36, 0xe4, 0x7b, 0xda, 0x63, 0x75, 0xcf, - 0xd5, 0x74, 0x91, 0xea, 0x75, 0xb9, 0x57, 0x88, 0x28, 0xb9, 0x9e, 0xf6, 0x78, 0x45, 0xd0, 0x9b, - 0x46, 0x17, 0x17, 0x7f, 0x1e, 0x81, 0xa4, 0xd8, 0x11, 0x72, 0x00, 0x84, 0x78, 0x4c, 0x83, 0x27, - 0x13, 0x99, 0xea, 0x47, 0xc7, 0x83, 0x52, 0xaa, 0xc6, 0xa8, 0xcd, 0x3a, 0x79, 0x36, 0x28, 0x7d, - 0xe7, 0x79, 0x9d, 0x96, 0x0f, 0xa2, 0xa4, 0xf8, 0x24, 0x4d, 0x83, 0x5d, 0x02, 0x1d, 0x68, 0x44, - 0x3d, 0x30, 0x89, 0x67, 0xef, 0xbb, 0x5a, 0x8f, 0x1d, 0xae, 0xa4, 0xcc, 0x1c, 0x68, 0x64, 0xcd, - 0xa7, 0xa1, 0x22, 0x4d, 0x07, 0x1e, 0xf2, 0x47, 0x5e, 0x6e, 0x52, 0x41, 0x1b, 0x2d, 0xc1, 0xb5, - 0x60, 0xb0, 0x4a, 0x37, 0xdd, 0xe9, 0xeb, 0x87, 0x98, 0x45, 0x02, 0xea, 0xb3, 0x66, 0x83, 0xce, - 0x0d, 0xed, 0x71, 0x95, 0x77, 0x95, 0xaf, 0xc1, 0x6c, 0xe8, 0x58, 0x83, 0x14, 0xeb, 0xbf, 0x01, - 0x92, 0xdb, 0xda, 0x51, 0xd7, 0xd6, 0x0c, 0xb4, 0x00, 0x69, 0xff, 0x71, 0x96, 0x26, 0xa9, 0x5c, - 0x0f, 0xc3, 0x24, 0x64, 0x42, 0xb6, 0x4f, 0xb0, 0x4b, 0xcf, 0x44, 0x65, 0x5f, 0xd4, 0x71, 0x5f, - 0x55, 0xad, 0x3e, 0x1b, 0x94, 0xee, 0x4e, 0x26, 0x22, 0xac, 0xf7, 0x5d, 0xd3, 0x3b, 0xaa, 0xb4, - 0x3f, 0xba, 0xb7, 0x2b, 0xa0, 0xa8, 0x21, 0xd9, 0x4a, 0xa6, 0x1f, 0x6e, 0x8a, 0xa7, 0x6e, 0xba, - 0x5d, 0xb5, 0x67, 0xea, 0xae, 0x4d, 0xfc, 0x1b, 0x51, 0x41, 0xdd, 0x60, 0x44, 0xf4, 0x1a, 0xe4, - 0xf6, 0x4c, 0x8b, 0xdd, 0xc6, 0xfb, 0x7c, 0xfc, 0x32, 0x34, 0xeb, 0x93, 0x05, 0xe3, 0x43, 0xc8, - 0x86, 0x9e, 0xb7, 0xe9, 0x51, 0x27, 0xd8, 0x51, 0x6f, 0x1d, 0x0f, 0x4a, 0x99, 0xa1, 0xe9, 0xf0, - 0xe3, 0xbe, 0x4a, 0x8c, 0xca, 0x0c, 0xa7, 0xa1, 0x87, 0x3d, 0x07, 0xd3, 0xec, 0x33, 0x49, 0xfe, - 0x45, 0x8b, 0xc2, 0x1b, 0xe8, 0x1d, 0x98, 0xee, 0x62, 0x8d, 0x60, 0xf1, 0xb1, 0xca, 0xc2, 0x39, - 0xc6, 0xc8, 0x3e, 0x58, 0x54, 0x38, 0x3b, 0xaa, 0x42, 0x82, 0xbf, 0xaf, 0xb0, 0x57, 0x91, 0xf4, - 0xd2, 0xcd, 0x49, 0xbf, 0x4b, 0x5a, 0x9b, 0x52, 0xc4, 0x48, 0xd4, 0x80, 0xa4, 0xcb, 0x9f, 0xd4, - 0xd8, 0x5b, 0xc9, 0x85, 0xe5, 0x6a, 0xe8, 0xc5, 0x6e, 0x6d, 0x4a, 0xf1, 0xc7, 0xa2, 0x1d, 0xff, - 0x5d, 0x9b, 0x7b, 0x75, 0xf1, 0x6c, 0x5f, 0x99, 0x30, 0x21, 0x1b, 0x02, 0x8e, 0xa0, 0xd0, 0x0d, - 0x9a, 0xec, 0x7e, 0x95, 0x3d, 0xb9, 0x9c, 0xbf, 0xc1, 0x91, 0xb7, 0x3a, 0xba, 0x41, 0x3e, 0x12, - 0x6d, 0x02, 0xe8, 0x41, 0xa4, 0x61, 0x8f, 0x31, 0xe9, 0xa5, 0x37, 0x2e, 0x93, 0xcd, 0xac, 0x4d, - 0x29, 0x21, 0x04, 0xf4, 0x11, 0xa4, 0xf5, 0xa1, 0xe9, 0x14, 0x72, 0x0c, 0xf0, 0xcd, 0x4b, 0xf9, - 0xcf, 0x35, 0xea, 0x33, 0x87, 0xd4, 0x51, 0x9f, 0x29, 0x8f, 0xfb, 0xcc, 0x06, 0x64, 0xc4, 0xcd, - 0x02, 0xff, 0xc2, 0xb6, 0x90, 0x67, 0x2e, 0x3b, 0xac, 0x25, 0xfe, 0x37, 0xb8, 0x95, 0x86, 0xa5, - 0xdb, 0x06, 0x36, 0x1a, 0xb4, 0xad, 0x88, 0xab, 0x4a, 0xd6, 0x20, 0x68, 0x15, 0xb2, 0x7a, 0x17, - 0x6b, 0x56, 0xdf, 0xf1, 0x71, 0xd0, 0x84, 0x38, 0x19, 0x31, 0x4e, 0x00, 0x6d, 0x02, 0xda, 0x63, - 0x6f, 0xe1, 0xe1, 0x55, 0xb1, 0x37, 0x9d, 0x49, 0xc0, 0x64, 0x36, 0x56, 0x19, 0xae, 0x0c, 0xbd, - 0x0c, 0x19, 0xcb, 0xb6, 0x74, 0xcd, 0xd2, 0x71, 0x97, 0x45, 0x37, 0xfe, 0x0c, 0x34, 0x4a, 0x44, - 0x9f, 0x42, 0x96, 0x8c, 0xa4, 0xf0, 0x85, 0x6b, 0x6c, 0xc6, 0xb7, 0x2e, 0x7b, 0x1f, 0xb8, 0x36, - 0xa5, 0x8c, 0x21, 0xa1, 0x5f, 0x07, 0xd9, 0x1b, 0xbb, 0x03, 0x66, 0x0f, 0x4a, 0xe7, 0x7f, 0x77, - 0x72, 0xc6, 0xa5, 0xf5, 0xda, 0x94, 0x72, 0x02, 0xad, 0x9a, 0x82, 0xa4, 0xb8, 0x31, 0x0f, 0xde, - 0xa1, 0x92, 0xb2, 0x54, 0xfe, 0x79, 0x02, 0xa4, 0x20, 0xf9, 0x59, 0x04, 0x14, 0x84, 0xab, 0xe1, - 0xc7, 0x3f, 0xd4, 0xef, 0x46, 0xd7, 0xa6, 0x94, 0xbc, 0xdf, 0x37, 0xfc, 0xfe, 0xe7, 0x35, 0xc8, - 0xf5, 0x6c, 0xc3, 0xdc, 0x33, 0x87, 0xde, 0x8e, 0xdf, 0x56, 0x65, 0x7d, 0xb2, 0xf0, 0x76, 0x77, - 0x47, 0xde, 0xc8, 0x63, 0x13, 0x84, 0xde, 0xb5, 0xa9, 0xd0, 0x23, 0x3a, 0xf5, 0xbe, 0x6e, 0xdf, - 0xb2, 0x4c, 0x6b, 0x5f, 0x15, 0xf5, 0x16, 0x0f, 0xec, 0x19, 0x41, 0x15, 0x25, 0x53, 0x6d, 0xcc, - 0x1d, 0xdd, 0xba, 0xd0, 0x1d, 0xf9, 0x7b, 0x5f, 0x8b, 0x04, 0xfe, 0x68, 0x65, 0xdc, 0x1f, 0xdd, - 0xbe, 0xd8, 0x1f, 0x85, 0x60, 0x02, 0x87, 0xb4, 0x7b, 0xaa, 0x43, 0x5a, 0x9c, 0x50, 0x5b, 0x42, - 0x88, 0xa3, 0x1e, 0xa9, 0x36, 0xe6, 0x91, 0x6e, 0x5d, 0xe8, 0x91, 0xc2, 0x7b, 0x14, 0x2e, 0x69, - 0xeb, 0x14, 0x97, 0xf4, 0xe6, 0x44, 0x2e, 0x29, 0x04, 0x16, 0xf6, 0x49, 0xca, 0x69, 0x3e, 0xa9, - 0x32, 0x99, 0x4f, 0x0a, 0x41, 0x8e, 0x38, 0xa5, 0xef, 0x9d, 0x30, 0x38, 0x99, 0xc1, 0x7e, 0xf3, - 0xd2, 0x45, 0xf6, 0x5a, 0xe4, 0x84, 0xc5, 0x69, 0xa7, 0x58, 0x5c, 0x9e, 0xc1, 0xbf, 0x7d, 0x09, - 0x8b, 0x0b, 0x4d, 0x70, 0xd2, 0xe4, 0x00, 0x24, 0xff, 0x2d, 0x2c, 0x64, 0x7e, 0xe5, 0x1f, 0x45, - 0x20, 0xd6, 0xb2, 0x3b, 0x28, 0x3b, 0xbc, 0x91, 0x60, 0x77, 0x09, 0x1f, 0x0e, 0xd9, 0x45, 0x7a, - 0xfd, 0xd2, 0x39, 0x2b, 0xf1, 0x67, 0x56, 0x82, 0x41, 0xe8, 0x7d, 0x48, 0x3a, 0x3c, 0x75, 0x12, - 0x16, 0x56, 0x3e, 0x6f, 0x3c, 0xe7, 0x54, 0xfc, 0x21, 0xb7, 0x6f, 0x85, 0xbf, 0x70, 0xdf, 0xb0, - 0x0d, 0x8c, 0xb2, 0x00, 0xdb, 0x1a, 0x21, 0xce, 0x81, 0xab, 0x11, 0x2c, 0x4f, 0xa1, 0x24, 0xc4, - 0xd6, 0x37, 0xda, 0x72, 0xe4, 0xf6, 0x77, 0xc3, 0x77, 0x14, 0x75, 0x65, 0xb9, 0xb9, 0xd9, 0xdc, - 0x5c, 0x55, 0x37, 0x97, 0x37, 0x1a, 0x6d, 0x79, 0x0a, 0x15, 0x60, 0xee, 0xe3, 0xe5, 0xe6, 0x8e, - 0xb8, 0xb4, 0x50, 0x9b, 0x9b, 0x3b, 0x0d, 0xe5, 0xfe, 0xf2, 0x3d, 0x39, 0x82, 0xae, 0x03, 0x52, - 0xb6, 0x6a, 0xeb, 0xed, 0x7a, 0x55, 0xad, 0x6d, 0x6d, 0x6c, 0x2f, 0xd7, 0x76, 0x9a, 0x5b, 0x9b, - 0x72, 0x14, 0x49, 0x10, 0xaf, 0x6f, 0x6d, 0x36, 0x64, 0xb8, 0xfd, 0xa3, 0x18, 0xc4, 0xa9, 0x8c, - 0xd1, 0xcb, 0x90, 0xde, 0xdd, 0x6c, 0x6f, 0x37, 0x6a, 0xcd, 0x95, 0x66, 0xa3, 0x2e, 0x4f, 0x15, - 0x67, 0x9f, 0x3c, 0x5d, 0xc8, 0xd1, 0xae, 0x5d, 0x8b, 0x38, 0x58, 0x67, 0xce, 0x05, 0x15, 0x21, - 0x51, 0x5d, 0xae, 0xad, 0xef, 0x6e, 0xcb, 0x91, 0x62, 0xf6, 0xc9, 0xd3, 0x05, 0xa0, 0x0c, 0xdc, - 0xb0, 0xd1, 0x0b, 0x90, 0x54, 0x1a, 0xed, 0x9d, 0x2d, 0xa5, 0x21, 0x47, 0x8b, 0xb9, 0x27, 0x4f, - 0x17, 0xd2, 0xb4, 0x53, 0xd8, 0x2b, 0x7a, 0x0d, 0x32, 0xed, 0xda, 0x5a, 0x63, 0x63, 0x59, 0xad, - 0xad, 0x2d, 0x6f, 0xae, 0x36, 0xe4, 0x58, 0x71, 0xee, 0xc9, 0xd3, 0x05, 0x79, 0xfc, 0x80, 0xe9, - 0x14, 0xcd, 0x8d, 0xed, 0x2d, 0x65, 0x47, 0x8e, 0x0f, 0xa7, 0xe0, 0x76, 0x85, 0xca, 0x00, 0x7c, - 0xf4, 0x4a, 0xa3, 0x51, 0x97, 0xa7, 0x8b, 0xe8, 0xc9, 0xd3, 0x85, 0x2c, 0xed, 0x1f, 0x9a, 0x0b, - 0x7a, 0x05, 0x66, 0x6a, 0x4a, 0x63, 0x79, 0xa7, 0xa1, 0xb6, 0x77, 0x96, 0x77, 0xda, 0x72, 0x62, - 0xb8, 0x93, 0x90, 0x09, 0xa0, 0x0a, 0xe4, 0x97, 0x77, 0x77, 0xb6, 0xd4, 0x11, 0xde, 0x64, 0xf1, - 0xc6, 0x93, 0xa7, 0x0b, 0xb3, 0x94, 0x77, 0xb9, 0xef, 0xd9, 0x61, 0xfe, 0x37, 0x40, 0x1e, 0x59, - 0xbf, 0xba, 0x5a, 0x93, 0xa5, 0xe2, 0xf5, 0x27, 0x4f, 0x17, 0xd0, 0xf8, 0x16, 0x56, 0x6b, 0xe8, - 0x57, 0xe0, 0xfa, 0xce, 0x27, 0xdb, 0x8d, 0x7a, 0xa3, 0x5d, 0x53, 0x47, 0xb7, 0x9d, 0x2a, 0x16, - 0x9e, 0x3c, 0x5d, 0x98, 0xa3, 0x63, 0xc6, 0xc7, 0x15, 0xa5, 0x1f, 0xfe, 0xe5, 0xfc, 0xd4, 0x5f, - 0xff, 0xd5, 0xfc, 0x54, 0xf5, 0xe6, 0x17, 0xff, 0x39, 0x3f, 0xf5, 0xc5, 0xf1, 0x7c, 0xe4, 0x27, - 0xc7, 0xf3, 0x91, 0x9f, 0x1e, 0xcf, 0x47, 0xfe, 0xe3, 0x78, 0x3e, 0xf2, 0x87, 0x5f, 0xce, 0x4f, - 0xfd, 0xe4, 0xcb, 0xf9, 0xa9, 0x9f, 0x7e, 0x39, 0x3f, 0xf5, 0x69, 0x82, 0xab, 0x55, 0x27, 0xc1, - 0xd2, 0xf0, 0xb7, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x90, 0x45, 0x7d, 0x1b, 0xc8, 0x33, 0x00, - 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_8868ebe50bd1b6b1) } + +var fileDescriptor_jobs_8868ebe50bd1b6b1 = []byte{ + // 4452 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x6c, 0x23, 0x47, + 0x7a, 0x56, 0x93, 0x14, 0xd9, 0xfc, 0x29, 0x52, 0xcd, 0x92, 0x66, 0x86, 0xcb, 0xd8, 0xa2, 0xc2, + 0xf5, 0x63, 0x66, 0x6c, 0x53, 0x5e, 0x39, 0xeb, 0xb5, 0x27, 0xf6, 0x78, 0xf9, 0x1a, 0x89, 0xd4, + 0xe8, 0xe1, 0xa6, 0x34, 0x5e, 0x7b, 0xe1, 0x74, 0x5a, 0xdd, 0x25, 0xa9, 0xa3, 0x66, 0x37, 0xa7, + 0xab, 0x39, 0x33, 0xda, 0x0d, 0x92, 0x60, 0x83, 0x04, 0x8b, 0x39, 0x25, 0x40, 0x36, 0x87, 0x24, + 0x03, 0x04, 0x48, 0x16, 0xc8, 0x21, 0x40, 0x00, 0x23, 0x48, 0x72, 0xc8, 0x6d, 0x2f, 0x3e, 0x24, + 0xc0, 0x5e, 0x02, 0x2c, 0x72, 0xe0, 0x26, 0xf2, 0x25, 0x97, 0x00, 0x8b, 0xe4, 0x36, 0xa7, 0xa0, + 0x1e, 0xdd, 0x6c, 0x52, 0x1a, 0x89, 0x1a, 0xd9, 0x9b, 0x8b, 0x86, 0xf5, 0xd7, 0x5f, 0x5f, 0x55, + 0xfd, 0xf5, 0x3f, 0xab, 0x7a, 0xe0, 0xea, 0x6f, 0xb9, 0xbb, 0x64, 0x89, 0xfe, 0xe9, 0xed, 0xb2, + 0x7f, 0x2a, 0x3d, 0xcf, 0xf5, 0x5d, 0xf4, 0x35, 0xc3, 0x35, 0x0e, 0x3d, 0x57, 0x37, 0x0e, 0x2a, + 0xe4, 0xbe, 0x5d, 0x61, 0x3d, 0x9c, 0xab, 0x78, 0x05, 0x7b, 0x9e, 0xeb, 0x51, 0x7e, 0xfe, 0x83, + 0x8f, 0x28, 0xce, 0xef, 0xbb, 0xfb, 0x2e, 0xfb, 0xb9, 0x44, 0x7f, 0x09, 0x2a, 0x62, 0x18, 0xbd, + 0xdd, 0x25, 0x53, 0xf7, 0x75, 0x41, 0x2b, 0x04, 0x34, 0xcb, 0x7d, 0x63, 0xcf, 0xf5, 0xba, 0xba, + 0x1f, 0x60, 0x7c, 0x9d, 0xdc, 0xb7, 0x97, 0x0c, 0xdd, 0xd7, 0x6d, 0x77, 0x7f, 0xc9, 0xc4, 0xc4, + 0xe8, 0xed, 0x2e, 0x11, 0xdf, 0xeb, 0x1b, 0x7e, 0xdf, 0xc3, 0xa6, 0x60, 0x2a, 0x9d, 0xc2, 0xe4, + 0x63, 0x47, 0x77, 0xfc, 0x00, 0xbf, 0xef, 0x5b, 0xf6, 0xd2, 0x81, 0x6d, 0x2c, 0xf9, 0x56, 0x17, + 0x13, 0x5f, 0xef, 0xf6, 0x78, 0x4f, 0xf9, 0x77, 0x61, 0xfa, 0x2e, 0xd6, 0x09, 0x46, 0x9f, 0x40, + 0xca, 0x71, 0x4d, 0xac, 0x59, 0x66, 0x41, 0x5a, 0x94, 0xae, 0x67, 0x6b, 0xd5, 0xe3, 0x41, 0x29, + 0xb9, 0xe1, 0x9a, 0xb8, 0xd5, 0x78, 0x3a, 0x28, 0xbd, 0xb5, 0x6f, 0xf9, 0x07, 0xfd, 0xdd, 0x8a, + 0xe1, 0x76, 0x97, 0x42, 0x41, 0x98, 0xbb, 0xc3, 0xdf, 0x4b, 0xbd, 0xc3, 0xfd, 0x25, 0xb1, 0x8d, + 0x0a, 0x1f, 0xa6, 0x26, 0x29, 0x62, 0xcb, 0x44, 0xf3, 0x30, 0x8d, 0x7b, 0xae, 0x71, 0x50, 0x88, + 0x2d, 0x4a, 0xd7, 0xe3, 0x2a, 0x6f, 0xdc, 0x4a, 0xfc, 0xd7, 0x5f, 0x96, 0xa4, 0xf2, 0x8f, 0x63, + 0x70, 0xad, 0xa6, 0x1b, 0x87, 0xfd, 0x5e, 0xd3, 0x31, 0xbc, 0xa3, 0x9e, 0x6f, 0xb9, 0xce, 0x26, + 0xfb, 0x4b, 0x90, 0x02, 0xf1, 0x43, 0x7c, 0xc4, 0xd6, 0x33, 0xa3, 0xd2, 0x9f, 0xe8, 0x7d, 0x48, + 0x74, 0x5d, 0x13, 0x33, 0xa0, 0xdc, 0xf2, 0x8d, 0xca, 0x33, 0xcf, 0xa4, 0x32, 0x44, 0x5b, 0x77, + 0x4d, 0xac, 0xb2, 0x61, 0x68, 0x17, 0xe4, 0xc3, 0x2e, 0xd1, 0x2c, 0x67, 0xcf, 0x2d, 0xc4, 0x17, + 0xa5, 0xeb, 0x99, 0xe5, 0x5b, 0x67, 0x40, 0x3c, 0x63, 0x59, 0x95, 0xb5, 0xf5, 0x4e, 0xcb, 0xd9, + 0x73, 0x6b, 0x99, 0xe3, 0x41, 0x29, 0x25, 0x1a, 0x6a, 0xea, 0xb0, 0x4b, 0xe8, 0x8f, 0xe2, 0x26, + 0x04, 0x34, 0xba, 0xfe, 0xbe, 0x67, 0xb1, 0xf5, 0xa7, 0x55, 0xfa, 0x13, 0xbd, 0x0e, 0x08, 0x73, + 0x3c, 0x6c, 0x6a, 0x54, 0x01, 0x34, 0xba, 0xc1, 0x18, 0xdb, 0xa0, 0x12, 0xf6, 0x34, 0x74, 0x5f, + 0x5f, 0xc3, 0x47, 0x5c, 0x42, 0x42, 0x4e, 0xbf, 0x17, 0x87, 0xdc, 0x70, 0x29, 0x0c, 0x7e, 0x15, + 0x92, 0xc4, 0x38, 0xc0, 0x5d, 0xcc, 0x66, 0xc8, 0x2d, 0xbf, 0x39, 0x91, 0x38, 0xe8, 0xd0, 0x4a, + 0x87, 0x8d, 0x53, 0xc5, 0x78, 0x84, 0x20, 0x41, 0x74, 0xdb, 0x17, 0x0b, 0x61, 0xbf, 0xd1, 0x9f, + 0x4b, 0xb0, 0x38, 0xbe, 0xa2, 0xda, 0xd1, 0xda, 0x7a, 0x67, 0x5d, 0x27, 0x3e, 0xf6, 0xd6, 0xf0, + 0x51, 0xab, 0x51, 0x88, 0x2f, 0xc6, 0xaf, 0x67, 0x96, 0x37, 0x27, 0x9f, 0xb8, 0x79, 0x0e, 0x62, + 0xd3, 0xf1, 0xbd, 0x23, 0xf5, 0xdc, 0x89, 0x8b, 0x1d, 0x78, 0x79, 0x22, 0xa8, 0xa8, 0x0e, 0xa5, + 0xb9, 0x0e, 0xcd, 0xc3, 0xf4, 0x03, 0xdd, 0xee, 0x63, 0xb1, 0x5b, 0xde, 0xb8, 0x15, 0x7b, 0x47, + 0x2a, 0x5f, 0x83, 0x24, 0x17, 0x0c, 0xca, 0x42, 0xba, 0xda, 0xec, 0x2c, 0x7f, 0xf3, 0xed, 0x95, + 0xfa, 0xba, 0x32, 0x25, 0x8e, 0xe0, 0x47, 0x12, 0x5c, 0xed, 0xf8, 0x1e, 0xd6, 0xbb, 0x2d, 0x67, + 0x1f, 0x13, 0xba, 0xa7, 0x06, 0xf6, 0x75, 0xcb, 0x26, 0xe8, 0xfb, 0x90, 0x23, 0xac, 0x47, 0xd3, + 0x4d, 0xd3, 0xc3, 0x84, 0xf0, 0x09, 0x6b, 0xdb, 0x4f, 0x07, 0xa5, 0xad, 0x89, 0x4c, 0xc7, 0x30, + 0xec, 0x25, 0x0e, 0x61, 0x39, 0xfb, 0xc3, 0x86, 0x61, 0x5b, 0xd8, 0xf1, 0x2b, 0x7c, 0xda, 0x2a, + 0xc7, 0x56, 0xb3, 0x24, 0xda, 0x2c, 0xff, 0x73, 0x12, 0xb2, 0x5c, 0x57, 0x83, 0xe5, 0xd4, 0x00, + 0x88, 0xaf, 0x7b, 0xbe, 0x46, 0xcd, 0x9d, 0x2d, 0x25, 0xb3, 0xfc, 0x62, 0xe4, 0x90, 0xa8, 0x3b, + 0xa8, 0x1c, 0xd8, 0x46, 0x65, 0x3b, 0x70, 0x07, 0xb5, 0xc4, 0xe7, 0x83, 0xd2, 0x94, 0x9a, 0x66, + 0xc3, 0x28, 0x15, 0xdd, 0x06, 0x19, 0x3b, 0x26, 0x47, 0x88, 0x4d, 0x8e, 0x90, 0xc2, 0x8e, 0xc9, + 0xc6, 0x7f, 0x8d, 0x2b, 0x7f, 0x9c, 0xc9, 0x21, 0x75, 0x3c, 0x28, 0xc5, 0x77, 0xd4, 0x16, 0xb7, + 0x82, 0xf7, 0xa0, 0x68, 0xe2, 0x9e, 0x87, 0x0d, 0x9d, 0x9a, 0xc1, 0x2e, 0x5b, 0xba, 0xd6, 0xd5, + 0x1d, 0x6b, 0x0f, 0x13, 0xbf, 0x90, 0x60, 0xc7, 0x52, 0x18, 0x72, 0xf0, 0xbd, 0xad, 0x8b, 0x7e, + 0xf4, 0xfb, 0x12, 0xcc, 0xf5, 0x3d, 0x8b, 0x68, 0xbb, 0x47, 0x9a, 0xed, 0x1a, 0xba, 0x6d, 0xf9, + 0x47, 0xda, 0xe1, 0x83, 0xc2, 0x34, 0xd3, 0xc5, 0xdb, 0xe7, 0x1a, 0xb4, 0x10, 0x52, 0x65, 0xc7, + 0xb3, 0x48, 0xed, 0xe8, 0xae, 0x40, 0x58, 0x7b, 0xc0, 0xf4, 0xa5, 0x36, 0x7f, 0x3c, 0x28, 0x29, + 0x3b, 0x6a, 0x2b, 0xda, 0x75, 0x4f, 0x55, 0xfa, 0x63, 0xcc, 0x48, 0x0f, 0x2d, 0xd9, 0x72, 0x1d, + 0xcd, 0xe5, 0xae, 0xa1, 0x90, 0x64, 0x82, 0x5a, 0xbe, 0xb8, 0x53, 0x51, 0xf3, 0xf8, 0x84, 0xfb, + 0xfb, 0x63, 0x09, 0x8a, 0xd4, 0x4b, 0x63, 0x83, 0x8a, 0x29, 0xf4, 0xdc, 0x9a, 0x87, 0x0d, 0xd7, + 0x33, 0x0b, 0x29, 0x2a, 0xa7, 0x5a, 0xe7, 0xdf, 0x27, 0x75, 0xce, 0x2c, 0x06, 0xf4, 0xfb, 0x96, + 0x59, 0xd9, 0xd9, 0x69, 0x35, 0x8e, 0x07, 0xa5, 0xc2, 0x56, 0x00, 0x1e, 0x1e, 0xa2, 0xca, 0xa0, + 0xd5, 0x42, 0xef, 0x19, 0x3d, 0xe8, 0x1d, 0xc8, 0x19, 0xae, 0x6d, 0x63, 0x83, 0x6d, 0x7b, 0x47, + 0x6d, 0x15, 0x64, 0x76, 0xc0, 0xf9, 0xe3, 0x41, 0x29, 0x5b, 0x0f, 0x7b, 0xe8, 0x51, 0x67, 0x8d, + 0x68, 0x13, 0xa9, 0x30, 0x1b, 0x11, 0x18, 0x73, 0xc1, 0x69, 0x26, 0xad, 0x1b, 0x13, 0x7b, 0x0f, + 0x35, 0x87, 0x47, 0xda, 0xc5, 0x3a, 0x5c, 0x39, 0xf5, 0x14, 0xcf, 0xb3, 0xfa, 0x74, 0xd4, 0xea, + 0x15, 0xc8, 0xf1, 0x43, 0xd9, 0xf2, 0xdc, 0x7d, 0x66, 0x50, 0x9f, 0xe5, 0x20, 0xa7, 0x62, 0xe2, + 0xbb, 0x1e, 0x0e, 0x2c, 0xea, 0x33, 0x09, 0xe6, 0x68, 0x64, 0xf5, 0xac, 0x9e, 0xef, 0x7a, 0x9a, + 0x87, 0x1f, 0x7a, 0x96, 0x8f, 0x49, 0x21, 0xc6, 0x94, 0xae, 0x7a, 0xc6, 0x16, 0x46, 0x81, 0x2a, + 0x8d, 0x10, 0x44, 0x15, 0x18, 0x5c, 0xef, 0x6e, 0xff, 0xe0, 0xe7, 0xa5, 0x5b, 0x13, 0x9d, 0xe3, + 0xc9, 0x60, 0x5f, 0x69, 0x35, 0x54, 0x64, 0x9e, 0x00, 0x46, 0x2f, 0x40, 0x82, 0xea, 0x2d, 0xf3, + 0xd2, 0xe9, 0x9a, 0x7c, 0x3c, 0x28, 0x25, 0xa8, 0x66, 0xab, 0x8c, 0x3a, 0x62, 0xe0, 0x89, 0xe7, + 0x30, 0xf0, 0x15, 0xc8, 0xf8, 0xfa, 0xae, 0x8d, 0x35, 0x3a, 0x33, 0x11, 0xe6, 0xf7, 0xca, 0x98, + 0x24, 0xc8, 0x7d, 0x7b, 0x57, 0x27, 0xb8, 0xb2, 0x4d, 0x39, 0x23, 0x7b, 0x07, 0x3f, 0x20, 0x10, + 0xb4, 0x04, 0x19, 0xf7, 0x01, 0xf6, 0x3c, 0xcb, 0xc4, 0x9a, 0xb9, 0xcb, 0x6c, 0x28, 0x5d, 0xcb, + 0x1d, 0x0f, 0x4a, 0xb0, 0x29, 0xc8, 0x8d, 0x9a, 0x0a, 0x01, 0x4b, 0x63, 0x17, 0xf9, 0x30, 0x2f, + 0x9c, 0x46, 0x68, 0xff, 0x4c, 0x9f, 0x52, 0x6c, 0x09, 0xef, 0x4d, 0x7e, 0x18, 0xfc, 0xdc, 0x03, + 0xe5, 0x61, 0x41, 0x9d, 0x6f, 0x12, 0xed, 0x9e, 0xe8, 0x41, 0xaf, 0x41, 0xbe, 0xe7, 0xe1, 0x9e, + 0xee, 0x61, 0xcd, 0x70, 0xbb, 0x3d, 0x1b, 0xfb, 0xd8, 0x64, 0xda, 0x2f, 0xab, 0x8a, 0xe8, 0xa8, + 0x07, 0x74, 0xf4, 0x32, 0x0d, 0x08, 0xba, 0x4f, 0x73, 0x0d, 0x82, 0x3d, 0xca, 0x99, 0x66, 0x9c, + 0x59, 0x46, 0x6d, 0x09, 0x22, 0x7a, 0x0b, 0xae, 0x0c, 0xcf, 0x8d, 0x68, 0xbd, 0xfe, 0xae, 0x6d, + 0x91, 0x03, 0x6c, 0x16, 0x80, 0x71, 0xcf, 0x47, 0x3a, 0xb7, 0x82, 0x3e, 0x74, 0x34, 0xa2, 0x8a, + 0x06, 0x15, 0x8c, 0xbe, 0x8f, 0x0b, 0x99, 0x45, 0xe9, 0xfa, 0x74, 0x6d, 0xf5, 0xe9, 0xa0, 0xd4, + 0x98, 0x58, 0x8f, 0x08, 0xee, 0x2e, 0xf9, 0x1e, 0xc6, 0x11, 0xb5, 0xac, 0x0b, 0xbc, 0xa8, 0x46, + 0x05, 0x34, 0xa4, 0x02, 0x0c, 0x4d, 0xb0, 0x30, 0xf3, 0xdc, 0xde, 0x2e, 0x82, 0x82, 0xaa, 0x90, + 0xe2, 0xc9, 0x2a, 0x29, 0x64, 0xd9, 0x01, 0xfe, 0xea, 0xb3, 0x74, 0x88, 0x71, 0x45, 0x4e, 0x29, + 0x18, 0x87, 0x1a, 0x00, 0xfe, 0x51, 0x2f, 0xd0, 0xc4, 0x1c, 0x43, 0x79, 0xf9, 0x59, 0x28, 0x47, + 0xbd, 0xa8, 0x22, 0xa6, 0x7d, 0xd1, 0x26, 0xa8, 0x0d, 0x33, 0x2c, 0x1f, 0xd2, 0x05, 0xce, 0x2c, + 0xc3, 0x79, 0xf5, 0x19, 0x38, 0x2c, 0x53, 0xd0, 0x23, 0x48, 0x19, 0x12, 0x52, 0x08, 0xda, 0x82, + 0x1c, 0x4d, 0xef, 0x28, 0xa7, 0x40, 0x53, 0x18, 0xda, 0x8d, 0x67, 0xa0, 0x35, 0x04, 0x73, 0x04, + 0x2f, 0x6b, 0x46, 0x68, 0xa4, 0xf8, 0xbf, 0x12, 0xe4, 0x4f, 0x38, 0x0f, 0xb4, 0x0d, 0xb1, 0x30, + 0x63, 0xa7, 0x3e, 0x3d, 0xc6, 0xb2, 0xf5, 0xcb, 0x38, 0x92, 0x98, 0x65, 0xa2, 0x7d, 0x48, 0x53, + 0x75, 0x76, 0x7c, 0x5a, 0x0e, 0xc4, 0x18, 0x78, 0xfb, 0x78, 0x50, 0x92, 0xb7, 0x18, 0xf1, 0xd2, + 0x53, 0xc8, 0x1c, 0xbc, 0x65, 0xa2, 0x12, 0x64, 0x7c, 0x57, 0xc3, 0x8f, 0x2c, 0xe2, 0x5b, 0xce, + 0x3e, 0x4b, 0x16, 0x64, 0x15, 0x7c, 0xb7, 0x29, 0x28, 0xc5, 0xbf, 0x88, 0x01, 0x3a, 0x69, 0xa5, + 0xe8, 0x9f, 0x24, 0x78, 0x21, 0xc8, 0x01, 0x5c, 0xcf, 0xda, 0xb7, 0x1c, 0xdd, 0x1e, 0x49, 0x06, + 0x24, 0x26, 0xed, 0x4f, 0x2e, 0xe3, 0x0a, 0x44, 0x82, 0xb0, 0x29, 0xe0, 0xc7, 0x13, 0x85, 0x17, + 0x68, 0x04, 0xe5, 0x89, 0xc2, 0x09, 0x96, 0x7b, 0x6a, 0xa1, 0xff, 0x8c, 0xc1, 0xc5, 0x35, 0x78, + 0xf1, 0x4c, 0xe0, 0x8b, 0xc4, 0xae, 0xe2, 0x0f, 0x24, 0xb8, 0xf6, 0x8c, 0x88, 0x12, 0xc5, 0xc9, + 0x72, 0x9c, 0x0f, 0xa3, 0x38, 0x99, 0xe5, 0x5f, 0xbf, 0x44, 0xd4, 0x8a, 0x2c, 0xa2, 0x9d, 0x90, + 0x25, 0x25, 0x56, 0x7e, 0x13, 0x66, 0xc5, 0xa0, 0x20, 0x8e, 0xa2, 0x17, 0x01, 0x0e, 0xac, 0xfd, + 0x03, 0xed, 0xa1, 0xee, 0x63, 0x4f, 0x94, 0x71, 0x69, 0x4a, 0xf9, 0x88, 0x12, 0xca, 0xff, 0x26, + 0x43, 0xb6, 0xd5, 0xed, 0xb9, 0x9e, 0x1f, 0x44, 0xd9, 0xbb, 0x90, 0x64, 0x71, 0x81, 0x88, 0xf3, + 0xab, 0x9c, 0xb1, 0xc2, 0x91, 0x91, 0x3c, 0xbe, 0x08, 0xb7, 0x20, 0x30, 0xc2, 0xf0, 0x17, 0x3b, + 0x35, 0xfc, 0xbd, 0x0f, 0x49, 0x5e, 0x6a, 0x8b, 0x4a, 0xb0, 0x14, 0x99, 0x2b, 0xa8, 0x63, 0x5b, + 0x9b, 0x77, 0x2c, 0x1b, 0xdf, 0x61, 0x6c, 0x01, 0x38, 0x1f, 0x84, 0x5e, 0x01, 0x99, 0x10, 0x5f, + 0x23, 0xd6, 0xf7, 0x78, 0xf4, 0x8c, 0xf3, 0x72, 0xb0, 0xd3, 0xd9, 0xee, 0x58, 0xdf, 0xc3, 0x6a, + 0x8a, 0x10, 0x9f, 0xfe, 0x40, 0x45, 0x90, 0x1f, 0xea, 0xb6, 0xcd, 0xa2, 0xec, 0x34, 0x2b, 0x7f, + 0xc3, 0xf6, 0xa8, 0x99, 0x25, 0xbf, 0x5a, 0x33, 0x13, 0x01, 0xb3, 0xa7, 0xfb, 0x07, 0x2c, 0x73, + 0x4c, 0xab, 0xc0, 0x49, 0x5b, 0xba, 0x7f, 0x80, 0x0a, 0x90, 0x22, 0x3a, 0x8d, 0x5d, 0xa4, 0x20, + 0x2f, 0xc6, 0xaf, 0xcf, 0xa8, 0x41, 0x13, 0x2d, 0x00, 0x8b, 0xbc, 0xbc, 0xc9, 0x82, 0x58, 0x5c, + 0x8d, 0x50, 0x98, 0x1c, 0x0e, 0xad, 0x9e, 0xb6, 0x77, 0x48, 0x78, 0xd0, 0x12, 0x72, 0x38, 0xb4, + 0x7a, 0x77, 0xd6, 0x88, 0x9a, 0xa2, 0x9d, 0x77, 0x0e, 0x09, 0x7a, 0x15, 0x66, 0x2d, 0x56, 0x35, + 0x69, 0xa6, 0xe5, 0x61, 0xc3, 0xb7, 0x8f, 0x58, 0xc0, 0x92, 0xd5, 0x1c, 0x27, 0x37, 0x04, 0x15, + 0xdd, 0x00, 0x65, 0x3c, 0xcc, 0xb2, 0x40, 0x23, 0xab, 0xb3, 0x63, 0x51, 0x96, 0xb2, 0xf2, 0xa3, + 0x8e, 0x04, 0xce, 0x2c, 0x67, 0xe5, 0xf4, 0x61, 0xcc, 0xac, 0xc0, 0x5c, 0x4f, 0xf7, 0x08, 0xd6, + 0x76, 0xfb, 0x8e, 0x69, 0x63, 0x8d, 0xfb, 0xea, 0x42, 0x8e, 0x71, 0xe7, 0x59, 0x57, 0x8d, 0xf5, + 0x70, 0xb7, 0x7e, 0x5e, 0xee, 0x7d, 0xf5, 0xff, 0x21, 0xf7, 0x2e, 0xfe, 0x38, 0x06, 0xd3, 0x4c, + 0xcf, 0xd1, 0x2d, 0x48, 0xd0, 0x63, 0x16, 0x95, 0xdd, 0xa4, 0x39, 0x17, 0x1b, 0x43, 0x6b, 0x7d, + 0x47, 0xef, 0xe2, 0x02, 0x62, 0x4a, 0xc0, 0x7e, 0xa3, 0x6b, 0x90, 0x22, 0xf8, 0xbe, 0xf6, 0x40, + 0xb7, 0x0b, 0x73, 0xec, 0x84, 0x93, 0x04, 0xdf, 0xbf, 0xa7, 0xdb, 0xe8, 0x0a, 0x24, 0x2d, 0xa2, + 0x39, 0xf8, 0x61, 0x61, 0x9e, 0x49, 0x6a, 0xda, 0x22, 0x1b, 0xf8, 0x21, 0x73, 0xdb, 0xba, 0xb7, + 0x8f, 0x7d, 0xcd, 0x70, 0x6d, 0x52, 0xb8, 0x42, 0x0d, 0x8c, 0xa6, 0x74, 0x94, 0x54, 0x77, 0x6d, + 0x82, 0x7e, 0x05, 0xd2, 0x0f, 0x75, 0xa2, 0xe1, 0x6e, 0xcf, 0x3f, 0x62, 0xc2, 0x92, 0xa9, 0xda, + 0x93, 0x26, 0x6d, 0xb7, 0x13, 0x72, 0x4c, 0x89, 0xb7, 0x13, 0x72, 0x5c, 0x49, 0xb4, 0x13, 0x72, + 0x42, 0x99, 0x6e, 0x27, 0xe4, 0x69, 0x25, 0xd9, 0x4e, 0xc8, 0x49, 0x25, 0xd5, 0x4e, 0xc8, 0x29, + 0x45, 0x6e, 0x27, 0x64, 0x59, 0x49, 0xb7, 0x13, 0x72, 0x5a, 0x81, 0x76, 0x42, 0x06, 0x25, 0xd3, + 0x4e, 0xc8, 0x19, 0x65, 0xa6, 0x9d, 0x90, 0x67, 0x94, 0x6c, 0x3b, 0x21, 0x67, 0x95, 0x5c, 0x3b, + 0x21, 0xe7, 0x94, 0xd9, 0x76, 0x42, 0x9e, 0x55, 0x94, 0x76, 0x42, 0x56, 0x94, 0x7c, 0x3b, 0x21, + 0xe7, 0x15, 0x54, 0xfe, 0x4c, 0x02, 0xa5, 0x83, 0xef, 0xf7, 0xb1, 0x63, 0xe0, 0x7b, 0xba, 0x5d, + 0x3f, 0xe8, 0x3b, 0x87, 0xe8, 0x15, 0x98, 0x35, 0xe8, 0x0f, 0x8d, 0x17, 0xc6, 0x74, 0xab, 0x12, + 0xdb, 0x6a, 0x96, 0x91, 0x3b, 0x94, 0x4a, 0x77, 0xfc, 0x22, 0x80, 0xe0, 0xa3, 0x96, 0xcd, 0x2f, + 0xac, 0xd2, 0x9c, 0x85, 0x9a, 0xf3, 0x18, 0x8c, 0xe7, 0x3e, 0x64, 0xee, 0x63, 0x04, 0x46, 0x75, + 0x1f, 0xa2, 0x25, 0x98, 0x77, 0xf0, 0x23, 0x5f, 0x1b, 0x67, 0x66, 0xae, 0x42, 0xcd, 0xd3, 0xbe, + 0x7a, 0x74, 0x40, 0xf9, 0x5f, 0x63, 0x30, 0x1b, 0x2c, 0x3a, 0x70, 0x87, 0x7b, 0xa0, 0xd0, 0x63, + 0xb1, 0x4c, 0xcd, 0x77, 0x39, 0x52, 0xe0, 0x18, 0xdf, 0x3f, 0xc3, 0x31, 0x8e, 0xa1, 0xd0, 0x76, + 0xcb, 0xdc, 0x76, 0xd9, 0x74, 0x3c, 0x34, 0xa8, 0x59, 0x12, 0xa5, 0x15, 0x77, 0x20, 0x17, 0x0c, + 0xe2, 0x14, 0x54, 0x87, 0xe4, 0xc8, 0x7c, 0xaf, 0x4d, 0x30, 0x5f, 0x20, 0x6a, 0x55, 0x0c, 0x2d, + 0x7e, 0x1f, 0xd0, 0xc9, 0xb9, 0xa3, 0x61, 0x69, 0x9a, 0x87, 0xa5, 0xcd, 0xd1, 0xb0, 0xf4, 0xee, + 0xc5, 0xf6, 0x16, 0x59, 0x76, 0xb4, 0xaa, 0xfb, 0x49, 0x0c, 0x72, 0x3c, 0x44, 0x84, 0xe1, 0xe8, + 0x35, 0xc8, 0x33, 0xa7, 0x65, 0x39, 0xfb, 0x5a, 0x4f, 0x10, 0xd9, 0xfe, 0x62, 0xaa, 0x12, 0x74, + 0x84, 0xcc, 0x5f, 0x87, 0xac, 0x87, 0x75, 0x73, 0xc8, 0x18, 0x63, 0x8c, 0x33, 0x94, 0x18, 0x32, + 0xbd, 0x0c, 0x39, 0x16, 0x0d, 0x87, 0x5c, 0x71, 0xc6, 0x95, 0x65, 0xd4, 0x90, 0xad, 0x06, 0x59, + 0xd2, 0xd3, 0x9d, 0x21, 0x57, 0x82, 0x09, 0xf5, 0xda, 0x29, 0x11, 0xa7, 0xd3, 0xd3, 0x1d, 0x11, + 0x69, 0x66, 0xe8, 0x98, 0x68, 0x2c, 0xf5, 0x30, 0xe9, 0x77, 0xb1, 0xd6, 0x73, 0x79, 0xb1, 0x15, + 0x57, 0xd3, 0x9c, 0xb2, 0xe5, 0x12, 0xb4, 0xc3, 0x54, 0x85, 0xc9, 0x42, 0x33, 0xb9, 0x70, 0x0a, + 0x49, 0x36, 0xcb, 0xcd, 0xc9, 0xc5, 0xa9, 0xce, 0x92, 0x51, 0x42, 0xf9, 0xb7, 0xe1, 0x1a, 0xcd, + 0x97, 0xb9, 0x53, 0xac, 0x1f, 0xe8, 0xce, 0x7e, 0xa8, 0x9c, 0x3a, 0xa4, 0x58, 0xce, 0x1d, 0xa6, + 0x9f, 0xab, 0xc7, 0x83, 0x52, 0x92, 0x72, 0x5f, 0x3a, 0x70, 0x25, 0x29, 0x70, 0xcb, 0x2c, 0x17, + 0xa1, 0x30, 0x3e, 0x7b, 0x58, 0xa3, 0xab, 0xac, 0x44, 0xef, 0x77, 0x31, 0x95, 0xd8, 0x5d, 0x8b, + 0xf8, 0xe8, 0xdb, 0x30, 0x23, 0x24, 0x44, 0x05, 0x17, 0x68, 0xee, 0x39, 0x42, 0xce, 0x78, 0x21, + 0x08, 0x29, 0xff, 0xbd, 0x04, 0x73, 0x0d, 0xcf, 0xed, 0xf5, 0xb0, 0x29, 0x7c, 0x27, 0xdf, 0x6a, + 0xe0, 0x32, 0xa5, 0x88, 0xcb, 0xdc, 0x80, 0x58, 0xab, 0x21, 0x72, 0xe3, 0xdb, 0x97, 0x4d, 0xb9, + 0x5b, 0x0d, 0xf4, 0x2e, 0x24, 0x69, 0x69, 0xd8, 0x27, 0xcc, 0x9f, 0xe4, 0x4e, 0x14, 0x41, 0x23, + 0xc7, 0xc6, 0x18, 0x55, 0x31, 0xa0, 0xfc, 0x87, 0x49, 0xb8, 0x12, 0x95, 0xd1, 0x4a, 0x3d, 0x58, + 0xf8, 0xa7, 0x90, 0xb2, 0x1c, 0x13, 0x3f, 0xc2, 0x13, 0xf9, 0x8d, 0xd3, 0x20, 0x2a, 0x42, 0x1e, + 0x2d, 0x0a, 0x13, 0x94, 0x5d, 0x02, 0x13, 0x7d, 0x27, 0x4c, 0xd7, 0xf8, 0x35, 0xc8, 0xad, 0xe7, + 0x46, 0x6f, 0x8c, 0xa5, 0x6e, 0x23, 0x99, 0x11, 0x73, 0xb0, 0x5f, 0x51, 0x66, 0xd4, 0x81, 0xbc, + 0xe5, 0xf8, 0xd8, 0xb3, 0xb1, 0xfe, 0x80, 0x06, 0x7a, 0x3a, 0xbd, 0xb8, 0x0d, 0x99, 0x34, 0xac, + 0x2a, 0x11, 0x00, 0x1e, 0x9e, 0x3f, 0x85, 0xb9, 0x28, 0x68, 0x70, 0x04, 0x67, 0xdf, 0x90, 0x30, + 0x09, 0x0f, 0x61, 0x83, 0x8b, 0x88, 0x08, 0x50, 0x8b, 0xe3, 0x14, 0xff, 0x54, 0x82, 0x99, 0xe8, + 0xb1, 0x20, 0x0b, 0x64, 0x36, 0x47, 0x60, 0x8b, 0xf1, 0xda, 0x06, 0xcd, 0xc1, 0x58, 0x27, 0x93, + 0xd5, 0x07, 0xcf, 0x2d, 0x2b, 0x0e, 0x21, 0x8e, 0xbc, 0x65, 0xd2, 0xc0, 0x6e, 0x7a, 0x6e, 0x6f, + 0x78, 0x2d, 0x1c, 0x57, 0x65, 0x4a, 0xa0, 0xb9, 0x4a, 0xf1, 0x77, 0x20, 0x1d, 0x1e, 0x68, 0xa4, + 0x32, 0x8d, 0x7f, 0x89, 0x95, 0xe9, 0x59, 0xf3, 0x97, 0x7f, 0x9e, 0x84, 0xb9, 0xd3, 0x5c, 0xd5, + 0xc7, 0xa0, 0x44, 0x3c, 0x83, 0x66, 0x5b, 0xc4, 0x17, 0x1a, 0x7b, 0xe3, 0xec, 0x12, 0x28, 0xe2, + 0x5e, 0xc4, 0x79, 0xe4, 0xbc, 0x51, 0xa7, 0xf3, 0x5d, 0xc8, 0x99, 0x7c, 0xcb, 0x9a, 0x30, 0x85, + 0xf8, 0xb9, 0x95, 0xcb, 0x29, 0x2e, 0x46, 0xa0, 0x67, 0xcd, 0x48, 0x17, 0x61, 0x37, 0xdd, 0x01, + 0x7a, 0x78, 0x9b, 0x60, 0x99, 0x4c, 0x3f, 0xb3, 0xb5, 0xce, 0xf1, 0xa0, 0x94, 0x17, 0x58, 0xc1, + 0xf5, 0xc1, 0xa5, 0x65, 0x9c, 0x37, 0xc7, 0x00, 0x4d, 0xea, 0xe8, 0x69, 0x3f, 0x9d, 0x78, 0x7a, + 0xe8, 0xe8, 0xa9, 0xa6, 0x5e, 0xde, 0xd1, 0xd3, 0x9f, 0x2d, 0x13, 0xfd, 0x81, 0x04, 0x79, 0x7e, + 0x97, 0xd8, 0xed, 0xfb, 0x3a, 0xbf, 0x20, 0x0e, 0x2a, 0xa2, 0x8f, 0x8f, 0x07, 0xa5, 0x59, 0x26, + 0x90, 0x75, 0xd1, 0xc7, 0xa6, 0xad, 0x3d, 0xef, 0xb4, 0x43, 0x14, 0x51, 0x25, 0x84, 0x04, 0x13, + 0xad, 0x41, 0x8e, 0x97, 0x77, 0x1a, 0x2d, 0x70, 0x2c, 0xd7, 0x61, 0xa5, 0x52, 0xb6, 0xf6, 0xd2, + 0xd3, 0x41, 0x69, 0xf1, 0x14, 0xcd, 0xe2, 0x95, 0xe1, 0x3d, 0xce, 0xab, 0x66, 0xf7, 0xa2, 0x4d, + 0x64, 0x40, 0x36, 0x54, 0x8d, 0xa3, 0x9e, 0xa8, 0xac, 0x2e, 0x1f, 0x2c, 0x66, 0x02, 0x1d, 0xa1, + 0x98, 0x68, 0x1f, 0x66, 0x83, 0x49, 0x78, 0x49, 0x43, 0x0a, 0xe9, 0x2f, 0x65, 0x9a, 0x40, 0xad, + 0xf9, 0xae, 0x89, 0x28, 0xf2, 0xaf, 0xc2, 0xfc, 0xa9, 0xd1, 0xf8, 0xcf, 0xa6, 0xe1, 0xea, 0xa8, + 0x87, 0x0f, 0x13, 0x17, 0x6d, 0x3c, 0x06, 0x7d, 0x30, 0x71, 0x94, 0x08, 0x30, 0xb8, 0x1b, 0x0a, + 0x5a, 0xe3, 0x51, 0xe8, 0xd3, 0xb1, 0x28, 0xf4, 0x1c, 0xf8, 0x4c, 0xbd, 0xc6, 0xf0, 0x05, 0x68, + 0xf1, 0x5f, 0x24, 0xc8, 0x8e, 0xcc, 0xff, 0xcb, 0x74, 0xb7, 0x5b, 0x61, 0x56, 0xc0, 0x5f, 0xbc, + 0xdf, 0xb9, 0xf8, 0xde, 0x46, 0x93, 0x85, 0xe2, 0x3f, 0x4a, 0x90, 0x1d, 0xd9, 0xee, 0x57, 0xe4, + 0xa8, 0xbf, 0xf4, 0x95, 0x97, 0xbf, 0x05, 0x49, 0x4e, 0x41, 0x08, 0x72, 0x1f, 0x55, 0x5b, 0xdb, + 0xad, 0x8d, 0x15, 0xed, 0xce, 0xa6, 0xaa, 0xad, 0xd4, 0x95, 0x29, 0x34, 0x03, 0x72, 0xa3, 0x79, + 0xb7, 0x49, 0x89, 0x8a, 0x84, 0x32, 0x90, 0x62, 0xad, 0x66, 0x43, 0x89, 0x95, 0x6b, 0xa0, 0x70, + 0xec, 0x3d, 0x4c, 0x5d, 0x2b, 0x2d, 0x52, 0x51, 0x05, 0xe6, 0x28, 0x2c, 0xee, 0xd2, 0x1c, 0x83, + 0x06, 0x13, 0x2d, 0x92, 0xe1, 0xe5, 0xc3, 0x2e, 0x1a, 0x56, 0x36, 0xf4, 0x2e, 0x2e, 0xff, 0x43, + 0x02, 0xf2, 0x43, 0x90, 0x20, 0xb0, 0xbc, 0x02, 0x32, 0xb1, 0x9c, 0x43, 0x6d, 0xf8, 0xd0, 0xc9, + 0x2f, 0x3f, 0x2c, 0xe7, 0x70, 0x47, 0x6d, 0xa9, 0x29, 0xda, 0xb9, 0xe3, 0x59, 0xa8, 0x0d, 0x09, + 0xb7, 0xe7, 0x07, 0x79, 0xff, 0xdb, 0x67, 0x88, 0xe2, 0xc4, 0x1c, 0x95, 0xcd, 0x9e, 0x2f, 0xaa, + 0x36, 0x86, 0x81, 0xfe, 0x56, 0x82, 0x14, 0xaf, 0xb4, 0x83, 0x0c, 0xff, 0xdd, 0x0b, 0xe1, 0x71, + 0x01, 0x88, 0x57, 0xa7, 0x8f, 0xa8, 0xaa, 0x3f, 0x1d, 0x94, 0xf2, 0xe3, 0x02, 0x22, 0x97, 0x7c, + 0x8e, 0x0a, 0x96, 0x88, 0xda, 0xfc, 0x21, 0x64, 0x28, 0x68, 0xe6, 0x52, 0x27, 0x7c, 0x6b, 0xca, + 0x8e, 0x1c, 0x44, 0x71, 0x1f, 0x66, 0xa2, 0xab, 0x3f, 0xe5, 0x86, 0xb3, 0x3a, 0x5a, 0x4a, 0xbe, + 0x36, 0x91, 0x64, 0x38, 0x66, 0xf4, 0x5a, 0xf5, 0x5b, 0x90, 0x0e, 0xc5, 0x7e, 0x91, 0xfb, 0x58, + 0xee, 0x25, 0xc3, 0x0b, 0x8e, 0x69, 0x25, 0x59, 0xfe, 0x3b, 0x09, 0x66, 0x54, 0x4c, 0x5c, 0xfb, + 0x01, 0x36, 0x69, 0xd6, 0x80, 0xbe, 0x01, 0x09, 0x9a, 0x85, 0x88, 0xbb, 0x9b, 0x73, 0xca, 0x13, + 0xc6, 0x8a, 0xaa, 0x90, 0x0e, 0x6f, 0xa0, 0x2e, 0xf2, 0x16, 0x3f, 0x1c, 0x85, 0x6e, 0x80, 0xb2, + 0xeb, 0xf6, 0x1d, 0x53, 0xf7, 0x8e, 0x34, 0x0f, 0xeb, 0xc6, 0x01, 0x36, 0xc5, 0x6d, 0xfb, 0x6c, + 0x40, 0x57, 0x39, 0xb9, 0xfc, 0xc3, 0x18, 0xa0, 0xa1, 0x70, 0x22, 0x6e, 0x82, 0xe6, 0x3e, 0x6c, + 0x1f, 0xa2, 0xc0, 0x8a, 0x9d, 0xfa, 0x3e, 0x32, 0x96, 0x42, 0x85, 0x1b, 0x0f, 0x8e, 0xd4, 0x8b, + 0xd0, 0x08, 0xfa, 0xd1, 0xd9, 0xf7, 0x6c, 0x71, 0x76, 0xcf, 0xc6, 0xb4, 0xf4, 0x97, 0x7a, 0xd7, + 0x26, 0xc2, 0xdd, 0xff, 0x24, 0x00, 0xd5, 0x3d, 0xac, 0xfb, 0x98, 0x7a, 0x1e, 0x72, 0x56, 0x3d, + 0x58, 0x83, 0x69, 0x5e, 0x3c, 0xc4, 0x2e, 0x52, 0x3c, 0x08, 0xa1, 0xf0, 0xa1, 0xe8, 0x37, 0x60, + 0xc6, 0x70, 0xed, 0x7e, 0xd7, 0xd1, 0xd8, 0x2b, 0xa1, 0x48, 0x25, 0xbf, 0x79, 0x96, 0x12, 0x9f, + 0x58, 0x5c, 0xa5, 0xee, 0xda, 0xb4, 0x1d, 0xd4, 0xb7, 0x1c, 0x90, 0x71, 0xa0, 0x17, 0x20, 0x1d, + 0x1a, 0x14, 0x4b, 0x22, 0xd3, 0xea, 0x90, 0x80, 0x96, 0x61, 0x5a, 0x27, 0x9a, 0xbb, 0xc7, 0xb2, + 0xbc, 0xf3, 0x34, 0x4c, 0x4d, 0xe8, 0x64, 0x73, 0x0f, 0xbd, 0x05, 0xd9, 0xbd, 0xfb, 0x3c, 0xf3, + 0xe5, 0x0e, 0x94, 0x3f, 0xde, 0xce, 0x1e, 0x0f, 0x4a, 0x99, 0x3b, 0x1f, 0xb2, 0xcd, 0x52, 0xf7, + 0xa9, 0x66, 0xf6, 0xee, 0x87, 0x0d, 0x74, 0x13, 0xf2, 0x5d, 0xfd, 0x91, 0xb6, 0xe7, 0xe9, 0x86, + 0x48, 0xf5, 0x6c, 0xee, 0x15, 0x24, 0x75, 0xb6, 0xab, 0x3f, 0xba, 0x23, 0xe8, 0x2d, 0xd3, 0xc6, + 0xc5, 0x5f, 0x48, 0x90, 0x12, 0x3b, 0x42, 0x3d, 0x00, 0x21, 0x1e, 0xcb, 0xe4, 0xc9, 0x44, 0xb6, + 0xf6, 0xe1, 0xf1, 0xa0, 0x94, 0xae, 0x33, 0x6a, 0xab, 0x41, 0x9e, 0x0e, 0x4a, 0xdf, 0x7e, 0x5e, + 0xa7, 0x15, 0x80, 0xa8, 0x69, 0x3e, 0x49, 0xcb, 0x64, 0x97, 0x40, 0x07, 0x3a, 0xd1, 0x0e, 0x2c, + 0xe2, 0xbb, 0xfb, 0x9e, 0xde, 0x65, 0x87, 0x2b, 0xab, 0x33, 0x07, 0x3a, 0x59, 0x0d, 0x68, 0xa8, + 0x48, 0xd3, 0x81, 0x07, 0xfc, 0x91, 0x97, 0x9b, 0x54, 0xd8, 0x46, 0xcb, 0x70, 0x25, 0x1c, 0xac, + 0xd1, 0x4d, 0xef, 0xf6, 0x8d, 0x43, 0xcc, 0x22, 0x01, 0xf5, 0x59, 0x73, 0x61, 0xe7, 0xba, 0xfe, + 0xa8, 0xc6, 0xbb, 0xca, 0x57, 0x60, 0x2e, 0x72, 0xac, 0x61, 0x8a, 0xf5, 0xdf, 0x00, 0xa9, 0x2d, + 0xfd, 0xc8, 0x76, 0x75, 0x13, 0x2d, 0x42, 0x26, 0x78, 0x9c, 0xa5, 0x49, 0x2a, 0xd7, 0xc3, 0x28, + 0x09, 0x59, 0x90, 0xeb, 0x13, 0xec, 0xd1, 0x33, 0xd1, 0xd8, 0x97, 0x7e, 0xdc, 0x57, 0xd5, 0x6a, + 0x4f, 0x07, 0xa5, 0xdb, 0x93, 0x89, 0x08, 0x1b, 0x7d, 0xcf, 0xf2, 0x8f, 0x2a, 0x9d, 0x0f, 0xef, + 0xee, 0x08, 0x28, 0x6a, 0x48, 0xae, 0x9a, 0xed, 0x47, 0x9b, 0xe2, 0xa9, 0x9b, 0x6e, 0x57, 0xeb, + 0x5a, 0x86, 0xe7, 0x92, 0xe0, 0x46, 0x54, 0x50, 0xd7, 0x19, 0x11, 0xbd, 0x0a, 0xb3, 0x7b, 0x96, + 0xc3, 0x6e, 0xe3, 0x03, 0x3e, 0x7e, 0x19, 0x9a, 0x0b, 0xc8, 0x82, 0xf1, 0x01, 0xe4, 0x22, 0xcf, + 0xdb, 0xf4, 0xa8, 0x93, 0xec, 0xa8, 0x37, 0x8f, 0x07, 0xa5, 0xec, 0xd0, 0x74, 0xf8, 0x71, 0x5f, + 0x26, 0x46, 0x65, 0x87, 0xd3, 0xd0, 0xc3, 0x9e, 0x87, 0x69, 0xf6, 0xf9, 0x26, 0xff, 0xa2, 0x45, + 0xe5, 0x0d, 0xf4, 0x36, 0x4c, 0xdb, 0x58, 0x27, 0x58, 0x7c, 0xac, 0xb2, 0x78, 0x86, 0x31, 0xb2, + 0x0f, 0x29, 0x55, 0xce, 0x8e, 0x6a, 0x90, 0xe4, 0xef, 0x2b, 0xec, 0x55, 0x24, 0xb3, 0x7c, 0x7d, + 0xd2, 0xef, 0x92, 0x56, 0xa7, 0x54, 0x31, 0x12, 0x35, 0x21, 0xe5, 0xf1, 0x27, 0x35, 0xf6, 0x56, + 0x72, 0x6e, 0xb9, 0x1a, 0x79, 0xb1, 0x5b, 0x9d, 0x52, 0x83, 0xb1, 0x68, 0x3b, 0x78, 0xd7, 0xe6, + 0x5e, 0x5d, 0x3c, 0xdb, 0x57, 0x26, 0x4c, 0xc8, 0x86, 0x80, 0x23, 0x28, 0x74, 0x83, 0x16, 0xbb, + 0x5f, 0x65, 0x4f, 0x2e, 0x67, 0x6f, 0x70, 0xe4, 0xad, 0x8e, 0x6e, 0x90, 0x8f, 0x44, 0x1b, 0x00, + 0x46, 0x18, 0x69, 0xd8, 0x63, 0x4c, 0x66, 0xf9, 0xf5, 0x8b, 0x64, 0x33, 0xab, 0x53, 0x6a, 0x04, + 0x01, 0x7d, 0x08, 0x19, 0x63, 0x68, 0x3a, 0x85, 0x59, 0x06, 0xf8, 0xc6, 0x85, 0xfc, 0xe7, 0x2a, + 0xf5, 0x99, 0x43, 0xea, 0xa8, 0xcf, 0x54, 0xc6, 0x7d, 0x66, 0x13, 0xb2, 0xe2, 0x66, 0x81, 0x7f, + 0xf9, 0x5b, 0xc8, 0x33, 0x97, 0x1d, 0xd5, 0x92, 0xe0, 0xdb, 0xe0, 0x4a, 0xd3, 0x31, 0x5c, 0x13, + 0x9b, 0x4d, 0xda, 0x56, 0xc5, 0x55, 0x25, 0x6b, 0x10, 0xb4, 0x02, 0x39, 0xc3, 0xc6, 0xba, 0xd3, + 0xef, 0x05, 0x38, 0x68, 0x42, 0x9c, 0xac, 0x18, 0x27, 0x80, 0x36, 0x00, 0xed, 0xb1, 0xb7, 0xf0, + 0xe8, 0xaa, 0xd8, 0x9b, 0xce, 0x24, 0x60, 0x0a, 0x1b, 0xab, 0x0e, 0x57, 0x86, 0x5e, 0x82, 0xac, + 0xe3, 0x3a, 0x86, 0xee, 0x18, 0xd8, 0x66, 0xd1, 0x8d, 0x3f, 0x03, 0x8d, 0x12, 0xd1, 0x27, 0x90, + 0x23, 0x23, 0x29, 0x7c, 0xe1, 0x0a, 0x9b, 0xf1, 0xcd, 0x8b, 0xde, 0x07, 0xae, 0x4e, 0xa9, 0x63, + 0x48, 0xe8, 0x37, 0x41, 0xf1, 0xc7, 0xee, 0x80, 0xd9, 0x83, 0xd2, 0xd9, 0xdf, 0x9d, 0x3c, 0xe3, + 0xd2, 0x7a, 0x75, 0x4a, 0x3d, 0x81, 0x56, 0x4b, 0x43, 0x4a, 0xdc, 0x98, 0x87, 0xef, 0x50, 0x29, + 0x45, 0x2e, 0xff, 0x22, 0x09, 0x72, 0x98, 0xfc, 0x2c, 0x01, 0x0a, 0xc3, 0xd5, 0xf0, 0xe3, 0x1f, + 0xea, 0x77, 0x63, 0xab, 0x53, 0x6a, 0x3e, 0xe8, 0x1b, 0x7e, 0xff, 0xf3, 0x2a, 0xcc, 0x76, 0x5d, + 0xd3, 0xda, 0xb3, 0x86, 0xde, 0x8e, 0xdf, 0x56, 0xe5, 0x02, 0xb2, 0xf0, 0x76, 0xb7, 0x47, 0xde, + 0xc8, 0xe3, 0x13, 0x84, 0xde, 0xd5, 0xa9, 0xc8, 0x23, 0x3a, 0xf5, 0xbe, 0x5e, 0xdf, 0x71, 0x2c, + 0x67, 0x5f, 0x13, 0xf5, 0x16, 0x0f, 0xec, 0x59, 0x41, 0x15, 0x25, 0x53, 0x7d, 0xcc, 0x1d, 0xdd, + 0x38, 0xd7, 0x1d, 0x05, 0x7b, 0x5f, 0x95, 0x42, 0x7f, 0x74, 0x67, 0xdc, 0x1f, 0xdd, 0x3c, 0xdf, + 0x1f, 0x45, 0x60, 0x42, 0x87, 0xb4, 0x73, 0xaa, 0x43, 0x5a, 0x9a, 0x50, 0x5b, 0x22, 0x88, 0xa3, + 0x1e, 0xa9, 0x3e, 0xe6, 0x91, 0x6e, 0x9c, 0xeb, 0x91, 0xa2, 0x7b, 0x14, 0x2e, 0x69, 0xf3, 0x14, + 0x97, 0xf4, 0xc6, 0x44, 0x2e, 0x29, 0x02, 0x16, 0xf5, 0x49, 0xea, 0x69, 0x3e, 0xa9, 0x32, 0x99, + 0x4f, 0x8a, 0x40, 0x8e, 0x38, 0xa5, 0xef, 0x9e, 0x30, 0x38, 0x85, 0xc1, 0x7e, 0xe3, 0xc2, 0x45, + 0xf6, 0xaa, 0x74, 0xc2, 0xe2, 0xf4, 0x53, 0x2c, 0x2e, 0xcf, 0xe0, 0xdf, 0xba, 0x80, 0xc5, 0x45, + 0x26, 0x38, 0x69, 0x72, 0x00, 0x72, 0xf0, 0x16, 0x16, 0x31, 0xbf, 0xf2, 0x9f, 0x48, 0x10, 0x6f, + 0xbb, 0xbb, 0x28, 0x37, 0xbc, 0x91, 0x60, 0x77, 0x09, 0x1f, 0x0c, 0xd9, 0x45, 0x7a, 0xfd, 0xf5, + 0x33, 0x56, 0x12, 0xcc, 0xac, 0x86, 0x83, 0xd0, 0x7b, 0x90, 0xea, 0xf1, 0xd4, 0x49, 0x58, 0x58, + 0xf9, 0xac, 0xf1, 0x9c, 0x53, 0x0d, 0x86, 0xdc, 0xbc, 0x11, 0xfd, 0xf2, 0x7e, 0xdd, 0x35, 0x31, + 0xca, 0x01, 0x6c, 0xe9, 0x84, 0xf4, 0x0e, 0x3c, 0x9d, 0x60, 0x65, 0x0a, 0xa5, 0x20, 0xbe, 0xb6, + 0xde, 0x51, 0xa4, 0x9b, 0xdf, 0x89, 0xde, 0x51, 0x34, 0xd4, 0x6a, 0x6b, 0xa3, 0xb5, 0xb1, 0xa2, + 0x6d, 0x54, 0xd7, 0x9b, 0x1d, 0x65, 0x0a, 0x15, 0x60, 0xfe, 0xa3, 0x6a, 0x6b, 0x5b, 0x5c, 0x5a, + 0x68, 0xad, 0x8d, 0xed, 0xa6, 0x7a, 0xaf, 0x7a, 0x57, 0x91, 0xd0, 0x55, 0x40, 0xea, 0x66, 0x7d, + 0xad, 0xd3, 0xa8, 0x69, 0xf5, 0xcd, 0xf5, 0xad, 0x6a, 0x7d, 0xbb, 0xb5, 0xb9, 0xa1, 0xc4, 0x90, + 0x0c, 0x89, 0xc6, 0xe6, 0x46, 0x53, 0x81, 0x9b, 0x3f, 0x89, 0x43, 0x82, 0xca, 0x18, 0xbd, 0x04, + 0x99, 0x9d, 0x8d, 0xce, 0x56, 0xb3, 0xde, 0xba, 0xd3, 0x6a, 0x36, 0x94, 0xa9, 0xe2, 0xdc, 0xe3, + 0x27, 0x8b, 0xb3, 0xb4, 0x6b, 0xc7, 0x21, 0x3d, 0x6c, 0x30, 0xe7, 0x82, 0x8a, 0x90, 0xac, 0x55, + 0xeb, 0x6b, 0x3b, 0x5b, 0x8a, 0x54, 0xcc, 0x3d, 0x7e, 0xb2, 0x08, 0x94, 0x81, 0x1b, 0x36, 0x7a, + 0x01, 0x52, 0x6a, 0xb3, 0xb3, 0xbd, 0xa9, 0x36, 0x95, 0x58, 0x71, 0xf6, 0xf1, 0x93, 0xc5, 0x0c, + 0xed, 0x14, 0xf6, 0x8a, 0x5e, 0x85, 0x6c, 0xa7, 0xbe, 0xda, 0x5c, 0xaf, 0x6a, 0xf5, 0xd5, 0xea, + 0xc6, 0x4a, 0x53, 0x89, 0x17, 0xe7, 0x1f, 0x3f, 0x59, 0x54, 0xc6, 0x0f, 0x98, 0x4e, 0xd1, 0x5a, + 0xdf, 0xda, 0x54, 0xb7, 0x95, 0xc4, 0x70, 0x0a, 0x6e, 0x57, 0xa8, 0x0c, 0xc0, 0x47, 0xdf, 0x69, + 0x36, 0x1b, 0xca, 0x74, 0x11, 0x3d, 0x7e, 0xb2, 0x98, 0xa3, 0xfd, 0x43, 0x73, 0x41, 0x2f, 0xc3, + 0x4c, 0x5d, 0x6d, 0x56, 0xb7, 0x9b, 0x5a, 0x67, 0xbb, 0xba, 0xdd, 0x51, 0x92, 0xc3, 0x9d, 0x44, + 0x4c, 0x00, 0x55, 0x20, 0x5f, 0xdd, 0xd9, 0xde, 0xd4, 0x46, 0x78, 0x53, 0xc5, 0x6b, 0x8f, 0x9f, + 0x2c, 0xce, 0x51, 0xde, 0x6a, 0xdf, 0x77, 0xa3, 0xfc, 0xaf, 0x83, 0x32, 0xb2, 0x7e, 0x6d, 0xa5, + 0xae, 0xc8, 0xc5, 0xab, 0x8f, 0x9f, 0x2c, 0xa2, 0xf1, 0x2d, 0xac, 0xd4, 0xd1, 0xaf, 0xc1, 0xd5, + 0xed, 0x8f, 0xb7, 0x9a, 0x8d, 0x66, 0xa7, 0xae, 0x8d, 0x6e, 0x3b, 0x5d, 0x2c, 0x3c, 0x7e, 0xb2, + 0x38, 0x4f, 0xc7, 0x9c, 0xd8, 0xfa, 0x1b, 0xa0, 0x74, 0xb6, 0xd5, 0x66, 0x75, 0x5d, 0x6b, 0x6d, + 0xac, 0x34, 0x3b, 0xec, 0xb0, 0x60, 0xb8, 0xa4, 0xb1, 0xff, 0x24, 0x50, 0x94, 0x7f, 0xf8, 0x57, + 0x0b, 0x53, 0x7f, 0xf3, 0xd7, 0x0b, 0x53, 0xb5, 0xeb, 0x9f, 0xff, 0xe7, 0xc2, 0xd4, 0xe7, 0xc7, + 0x0b, 0xd2, 0x4f, 0x8f, 0x17, 0xa4, 0x9f, 0x1d, 0x2f, 0x48, 0xff, 0x71, 0xbc, 0x20, 0xfd, 0xd1, + 0x17, 0x0b, 0x53, 0x3f, 0xfd, 0x62, 0x61, 0xea, 0x67, 0x5f, 0x2c, 0x4c, 0x7d, 0x92, 0xe4, 0x5a, + 0xb8, 0x9b, 0x64, 0x59, 0xfb, 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x18, 0xc0, 0x43, 0xa9, + 0x8f, 0x34, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 5c7f7e03ce8f..c004bac4bc66 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -77,6 +77,11 @@ message EncryptionInfo { map encryptedDataKeyByKMSMasterKeyID = 3; } +message StreamIngestionDetails { + // StreamAddress is the location of the stream which the ingestion job will + // read from. + string stream_address = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.StreamAddress"]; +} message BackupDetails { util.hlc.Timestamp start_time = 1 [(gogoproto.nullable) = false]; @@ -601,6 +606,7 @@ enum Type { // We can't name this TYPE_SCHEMA_CHANGE due to how proto generates actual // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TYPEDESC_SCHEMA_CHANGE = 9 [(gogoproto.enumvalue_customname) = "TypeTypeSchemaChange"]; + STREAM_INGESTION = 10 [(gogoproto.enumvalue_customname) = "TypeStreamIngestion"]; } message Job { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 0c0ef06ec16e..ed0513a29286 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -220,7 +220,7 @@ const ( func (Type) SafeValue() {} // NumJobTypes is the number of jobs types. -const NumJobTypes = 10 +const NumJobTypes = 11 func init() { if len(Type_name) != NumJobTypes { diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index f9376021f163..b46a4b7980ae 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -47,6 +47,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/ccl/streamingccl/streamclient", "//pkg/jobs/jobspb", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/sql/execinfrapb/processors.pb.go b/pkg/sql/execinfrapb/processors.pb.go index 02b50a6fee55..1d939e464e8f 100644 --- a/pkg/sql/execinfrapb/processors.pb.go +++ b/pkg/sql/execinfrapb/processors.pb.go @@ -77,7 +77,7 @@ func (m *ProcessorSpec) Reset() { *m = ProcessorSpec{} } func (m *ProcessorSpec) String() string { return proto.CompactTextString(m) } func (*ProcessorSpec) ProtoMessage() {} func (*ProcessorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{0} + return fileDescriptor_processors_29d19df42cdc68ae, []int{0} } func (m *ProcessorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -133,13 +133,14 @@ type ProcessorCoreUnion struct { SplitAndScatter *SplitAndScatterSpec `protobuf:"bytes,32,opt,name=splitAndScatter" json:"splitAndScatter,omitempty"` RestoreData *RestoreDataSpec `protobuf:"bytes,33,opt,name=restoreData" json:"restoreData,omitempty"` Filterer *FiltererSpec `protobuf:"bytes,34,opt,name=filterer" json:"filterer,omitempty"` + StreamIngestionData *StreamIngestionDataSpec `protobuf:"bytes,35,opt,name=streamIngestionData" json:"streamIngestionData,omitempty"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } func (m *ProcessorCoreUnion) String() string { return proto.CompactTextString(m) } func (*ProcessorCoreUnion) ProtoMessage() {} func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{1} + return fileDescriptor_processors_29d19df42cdc68ae, []int{1} } func (m *ProcessorCoreUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -174,7 +175,7 @@ func (m *NoopCoreSpec) Reset() { *m = NoopCoreSpec{} } func (m *NoopCoreSpec) String() string { return proto.CompactTextString(m) } func (*NoopCoreSpec) ProtoMessage() {} func (*NoopCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{2} + return fileDescriptor_processors_29d19df42cdc68ae, []int{2} } func (m *NoopCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -216,7 +217,7 @@ func (m *LocalPlanNodeSpec) Reset() { *m = LocalPlanNodeSpec{} } func (m *LocalPlanNodeSpec) String() string { return proto.CompactTextString(m) } func (*LocalPlanNodeSpec) ProtoMessage() {} func (*LocalPlanNodeSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{3} + return fileDescriptor_processors_29d19df42cdc68ae, []int{3} } func (m *LocalPlanNodeSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -249,7 +250,7 @@ func (m *MetadataTestSenderSpec) Reset() { *m = MetadataTestSenderSpec{} func (m *MetadataTestSenderSpec) String() string { return proto.CompactTextString(m) } func (*MetadataTestSenderSpec) ProtoMessage() {} func (*MetadataTestSenderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{4} + return fileDescriptor_processors_29d19df42cdc68ae, []int{4} } func (m *MetadataTestSenderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -282,7 +283,7 @@ func (m *MetadataTestReceiverSpec) Reset() { *m = MetadataTestReceiverSp func (m *MetadataTestReceiverSpec) String() string { return proto.CompactTextString(m) } func (*MetadataTestReceiverSpec) ProtoMessage() {} func (*MetadataTestReceiverSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_7bc79cdd8886d88f, []int{5} + return fileDescriptor_processors_29d19df42cdc68ae, []int{5} } func (m *MetadataTestReceiverSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -742,6 +743,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n32 } + if m.StreamIngestionData != nil { + dAtA[i] = 0x9a + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.StreamIngestionData.Size())) + n33, err := m.StreamIngestionData.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n33 + } return i, nil } @@ -1014,6 +1027,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.Filterer.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.StreamIngestionData != nil { + l = m.StreamIngestionData.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -1169,6 +1186,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.Filterer != nil { return this.Filterer } + if this.StreamIngestionData != nil { + return this.StreamIngestionData + } return nil } @@ -1234,6 +1254,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.RestoreData = vt case *FiltererSpec: this.Filterer = vt + case *StreamIngestionDataSpec: + this.StreamIngestionData = vt default: return false } @@ -2499,6 +2521,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 35: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamIngestionData", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StreamIngestionData == nil { + m.StreamIngestionData = &StreamIngestionDataSpec{} + } + if err := m.StreamIngestionData.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -2951,85 +3006,87 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_7bc79cdd8886d88f) + proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_29d19df42cdc68ae) } -var fileDescriptor_processors_7bc79cdd8886d88f = []byte{ - // 1214 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x97, 0xd1, 0x52, 0x1b, 0x37, - 0x14, 0x86, 0x31, 0x18, 0x6c, 0xcb, 0x36, 0x38, 0x0a, 0x49, 0x55, 0xb7, 0x35, 0xd4, 0x93, 0xb4, - 0x6e, 0x4a, 0x4c, 0x87, 0xe9, 0xf4, 0x22, 0xd3, 0x4e, 0x1b, 0xdb, 0x65, 0x58, 0x9a, 0x12, 0xba, - 0xa6, 0x61, 0x86, 0x1b, 0x57, 0xec, 0x0a, 0xb3, 0x61, 0xbd, 0x5a, 0x24, 0x19, 0x42, 0xfa, 0x12, - 0x7d, 0x84, 0x3e, 0x0e, 0x57, 0x9d, 0x5c, 0x75, 0x72, 0xc5, 0xb4, 0xe6, 0x45, 0x32, 0x92, 0xd6, - 0xbb, 0x6b, 0x83, 0xbd, 0xdc, 0x30, 0xb6, 0xfc, 0xff, 0x9f, 0x8e, 0x8e, 0x8e, 0xa4, 0x03, 0x58, - 0xe5, 0xa7, 0xee, 0x3a, 0x79, 0x43, 0x2c, 0xc7, 0x3b, 0x62, 0xd8, 0x3f, 0x5c, 0xf7, 0x19, 0xb5, - 0x08, 0xe7, 0x94, 0xf1, 0xba, 0xcf, 0xa8, 0xa0, 0x10, 0x59, 0xd4, 0x3a, 0x61, 0x14, 0x5b, 0xc7, - 0x75, 0x7e, 0xea, 0xd6, 0x6d, 0x87, 0x0b, 0x7e, 0xea, 0xb2, 0xbe, 0x57, 0x2e, 0x8f, 0x7b, 0x6d, - 0x2c, 0xb0, 0x76, 0x95, 0x1f, 0x4f, 0xe6, 0x76, 0x0e, 0x31, 0x27, 0x81, 0xec, 0xd1, 0x14, 0x99, - 0x9c, 0x4d, 0xab, 0x6a, 0xd3, 0x60, 0x7d, 0xf7, 0xa4, 0xe3, 0xd0, 0x40, 0xb9, 0x36, 0x45, 0x69, - 0x1d, 0x63, 0xaf, 0x4b, 0x8e, 0x08, 0xb1, 0xf9, 0x1d, 0xd4, 0x02, 0x1f, 0xba, 0xa4, 0xc3, 0x05, - 0x16, 0x43, 0xf5, 0x03, 0xa9, 0x16, 0x17, 0x3e, 0xe1, 0xfa, 0x6f, 0x30, 0xbc, 0xdc, 0xa5, 0x5d, - 0xaa, 0x3e, 0xae, 0xcb, 0x4f, 0x7a, 0xb4, 0xfa, 0xcf, 0x1c, 0x28, 0xee, 0x0e, 0x69, 0x6d, 0x9f, - 0x58, 0xb0, 0x09, 0xe6, 0x1d, 0xcf, 0xef, 0x0b, 0x94, 0x5a, 0x9d, 0xab, 0xe5, 0x37, 0xbe, 0xac, - 0x4f, 0xca, 0x6b, 0xdd, 0x90, 0xb2, 0xf6, 0x85, 0x67, 0x49, 0x5f, 0x23, 0x7d, 0x79, 0xb5, 0x32, - 0x63, 0x6a, 0x2f, 0xdc, 0x04, 0x69, 0x8b, 0x32, 0x82, 0x66, 0x57, 0x53, 0xb5, 0xfc, 0xc6, 0xda, - 0x64, 0x46, 0x38, 0x77, 0x93, 0x32, 0xf2, 0xbb, 0xe7, 0x50, 0x2f, 0x00, 0x29, 0x3f, 0xdc, 0x02, - 0x0b, 0xb4, 0x2f, 0x64, 0x34, 0x73, 0x2a, 0x9a, 0x27, 0x93, 0x49, 0x2f, 0x95, 0xce, 0xa4, 0x7d, - 0x41, 0x58, 0x2c, 0xa0, 0xc0, 0x0f, 0x9b, 0x20, 0xed, 0x53, 0x2e, 0x50, 0x5a, 0x45, 0xf4, 0xd5, - 0x94, 0x88, 0x28, 0x17, 0x41, 0x54, 0x31, 0x8c, 0x32, 0xc3, 0x27, 0x20, 0xcb, 0x05, 0xee, 0x92, - 0x8e, 0x63, 0xa3, 0xf9, 0xd5, 0x54, 0x6d, 0xbe, 0xb1, 0x24, 0x7f, 0x1d, 0x5c, 0xad, 0x64, 0xda, - 0x72, 0xdc, 0x68, 0x99, 0x19, 0x25, 0x30, 0x6c, 0xf8, 0x1d, 0x28, 0x84, 0xdb, 0x24, 0xf5, 0x0b, - 0x4a, 0x7f, 0x3f, 0xd0, 0xe7, 0xc3, 0x85, 0x1b, 0x2d, 0x33, 0x1f, 0x0a, 0x0d, 0x1b, 0xfe, 0x00, - 0x0a, 0x8c, 0xf0, 0xbe, 0x2b, 0x3a, 0x6a, 0xf7, 0x50, 0x46, 0x2d, 0xbc, 0x3c, 0x16, 0x30, 0x27, - 0xbd, 0xba, 0xde, 0xdd, 0x3d, 0x33, 0xaf, 0xf5, 0x7b, 0xf2, 0x6b, 0xf5, 0x5f, 0x08, 0xe0, 0xcd, - 0xa4, 0xc2, 0x67, 0x20, 0xed, 0x51, 0xea, 0xa3, 0x94, 0x5a, 0xfe, 0x17, 0x93, 0x97, 0xbf, 0x43, - 0xa9, 0x2f, 0x6d, 0x72, 0xed, 0xa6, 0xf2, 0xc0, 0x5f, 0x40, 0x5e, 0x55, 0x99, 0x49, 0xb0, 0x4d, - 0x58, 0xb0, 0xa7, 0x53, 0x32, 0xb8, 0x17, 0x89, 0x15, 0x25, 0xee, 0x86, 0x5b, 0x00, 0xbc, 0xa6, - 0x8e, 0x17, 0xb0, 0xe6, 0x14, 0xab, 0x36, 0x99, 0xb5, 0x1d, 0x6a, 0x15, 0x2a, 0xe6, 0x85, 0xdf, - 0x83, 0x05, 0x4e, 0x99, 0x20, 0x2c, 0xd8, 0xd3, 0x47, 0x93, 0x29, 0x6d, 0xa5, 0x53, 0x84, 0xc0, - 0x23, 0xe3, 0xc0, 0xdd, 0x2e, 0x23, 0x5d, 0x2c, 0x28, 0x53, 0x9b, 0x39, 0x35, 0x8e, 0xe7, 0xa1, - 0x56, 0xc7, 0x11, 0x79, 0x61, 0x03, 0x64, 0xa5, 0xd0, 0xf1, 0x2c, 0x81, 0x32, 0x49, 0xe9, 0x6d, - 0x05, 0x4a, 0x45, 0x09, 0x7d, 0x32, 0xc5, 0x3d, 0xc2, 0xba, 0x44, 0x2e, 0x97, 0x30, 0x94, 0x4d, - 0x4a, 0xf1, 0xaf, 0x91, 0x58, 0xa7, 0x38, 0xe6, 0x96, 0x4b, 0x3b, 0xc6, 0xfc, 0x38, 0x60, 0xe5, - 0x92, 0x96, 0xb6, 0x15, 0x6a, 0xf5, 0xd2, 0x22, 0x2f, 0xfc, 0x09, 0x2c, 0x9c, 0x61, 0xb7, 0x4f, - 0x38, 0x02, 0x49, 0x94, 0x57, 0x4a, 0x17, 0x56, 0x4e, 0xe0, 0x93, 0xb1, 0x1c, 0x62, 0xeb, 0xe4, - 0xc8, 0x71, 0x5d, 0xc2, 0x50, 0x3e, 0x89, 0xd2, 0x08, 0xb5, 0x3a, 0x96, 0xc8, 0x0b, 0x5f, 0x00, - 0xc0, 0x08, 0xb6, 0x8d, 0x9e, 0x4f, 0x99, 0x40, 0xc5, 0xa4, 0x8b, 0xc5, 0x0c, 0xb5, 0x2d, 0x2c, - 0xb0, 0xa6, 0x45, 0x7e, 0xf8, 0x23, 0xc8, 0xb4, 0x71, 0xcf, 0x97, 0x41, 0x2d, 0x29, 0xd4, 0xe3, - 0x29, 0xd5, 0xa3, 0x85, 0x8a, 0x31, 0x74, 0xc1, 0x03, 0x50, 0xd2, 0x1f, 0xa3, 0xca, 0x40, 0x25, - 0x45, 0xaa, 0x27, 0x91, 0xc6, 0x6a, 0xe9, 0x06, 0x07, 0xfe, 0x01, 0x60, 0x8f, 0x08, 0x2c, 0x9f, - 0xa9, 0x3d, 0xc2, 0x45, 0x9b, 0x78, 0xf2, 0xac, 0x40, 0x45, 0xff, 0x66, 0x5a, 0x51, 0x8c, 0x7b, - 0x14, 0xff, 0x16, 0x16, 0x3c, 0x02, 0xcb, 0xf1, 0x51, 0x93, 0x58, 0xc4, 0x39, 0x23, 0x0c, 0xdd, - 0x57, 0x73, 0x6c, 0xdc, 0x6d, 0x8e, 0xa1, 0x4b, 0xcd, 0x72, 0x2b, 0x0f, 0xfe, 0x0c, 0x72, 0xcd, - 0xf6, 0xab, 0x7d, 0xe6, 0xc8, 0x63, 0xba, 0xac, 0xe0, 0x53, 0x1e, 0x94, 0x50, 0xaa, 0x88, 0x91, - 0x13, 0xee, 0x80, 0xc2, 0x5b, 0xa7, 0xfb, 0x16, 0x77, 0x83, 0x9a, 0x7e, 0xa0, 0x48, 0x53, 0x1e, - 0x83, 0x83, 0x98, 0x5a, 0xc1, 0x46, 0xfc, 0xb2, 0x2a, 0x7d, 0x46, 0x5f, 0x13, 0x4b, 0xb4, 0x89, - 0x40, 0x0f, 0x93, 0xaa, 0x72, 0x37, 0xd4, 0xea, 0x3a, 0x8a, 0xbc, 0xf2, 0xf0, 0x9f, 0x3b, 0x9e, - 0x4d, 0xcf, 0x09, 0x43, 0x1f, 0x25, 0x1d, 0xfe, 0xfd, 0x40, 0xa9, 0x0f, 0xff, 0xd0, 0x07, 0x7f, - 0x03, 0x45, 0x97, 0x5a, 0xd8, 0xdd, 0x75, 0xb1, 0xb7, 0x43, 0x6d, 0x82, 0x90, 0x02, 0x7d, 0x3d, - 0x19, 0xf4, 0x22, 0x2e, 0x57, 0xb4, 0x51, 0x82, 0xac, 0x4e, 0xdd, 0x46, 0xc4, 0xaa, 0xf3, 0xe3, - 0xa4, 0xea, 0x6c, 0x8e, 0x39, 0x74, 0x75, 0x8e, 0x73, 0xe0, 0x1e, 0x58, 0xd4, 0x63, 0x9b, 0x8c, - 0x7a, 0xc2, 0x21, 0x0c, 0x95, 0x93, 0x0e, 0x63, 0x73, 0x44, 0xaf, 0xb8, 0x63, 0x0c, 0xb9, 0x25, - 0x94, 0xd9, 0x8e, 0x87, 0x5d, 0x47, 0x5c, 0xa0, 0x4f, 0x92, 0xb6, 0xe4, 0x65, 0xa8, 0xd5, 0x5b, - 0x12, 0x79, 0x65, 0x3a, 0x65, 0xb3, 0x65, 0xd2, 0xf3, 0xa0, 0xee, 0x3e, 0x4d, 0x4a, 0x67, 0x23, - 0x2e, 0xd7, 0xe9, 0x1c, 0x21, 0xc8, 0x74, 0x3a, 0xde, 0x19, 0x61, 0x82, 0xd8, 0x9b, 0x8e, 0x2b, - 0x08, 0x23, 0x0c, 0x7d, 0x96, 0x94, 0x4e, 0x63, 0xcc, 0xa1, 0xd3, 0x39, 0xce, 0x91, 0xe9, 0x1c, - 0x8e, 0x05, 0xd5, 0x5d, 0x49, 0x4a, 0xa7, 0x31, 0xa2, 0xd7, 0xe9, 0x1c, 0x65, 0x0c, 0xef, 0xdd, - 0xbe, 0x2f, 0x6f, 0x3f, 0xb4, 0x72, 0x97, 0x7b, 0x57, 0x6b, 0xa3, 0x7b, 0x57, 0x7f, 0x87, 0xfb, - 0x60, 0x89, 0xfb, 0xae, 0x23, 0x9e, 0x7b, 0x76, 0xdb, 0xc2, 0x42, 0x26, 0x74, 0x55, 0xe1, 0x9e, - 0x4e, 0xb9, 0xe7, 0x46, 0x0d, 0x8a, 0x39, 0x4e, 0x91, 0x6f, 0x1e, 0x23, 0x5c, 0x50, 0x46, 0x54, - 0x8c, 0x9f, 0x27, 0xbd, 0x79, 0x66, 0x24, 0xd6, 0x6f, 0x5e, 0xcc, 0x2d, 0xcf, 0xe1, 0xd1, 0x70, - 0x67, 0xaa, 0x49, 0xe7, 0x70, 0x64, 0x47, 0x42, 0xdf, 0xb3, 0xf4, 0xe5, 0xdf, 0x2b, 0xa9, 0xed, - 0x74, 0x76, 0xb1, 0xb4, 0xb4, 0x9d, 0xce, 0xde, 0x2b, 0xc1, 0xed, 0x74, 0x76, 0xa1, 0x94, 0xd9, - 0x4e, 0x67, 0x0b, 0xa5, 0x62, 0x75, 0x11, 0x14, 0xe2, 0xbd, 0x51, 0xf5, 0x4f, 0x70, 0xef, 0xc6, - 0x31, 0x84, 0x35, 0x50, 0x30, 0xe9, 0x79, 0x9b, 0xf6, 0x99, 0x45, 0x0c, 0xfb, 0x8d, 0x6a, 0xb7, - 0x8a, 0x41, 0x0b, 0x39, 0xf2, 0x0b, 0xac, 0x82, 0xdc, 0x4e, 0xbf, 0xa7, 0x5a, 0x68, 0xae, 0x5a, - 0xaa, 0xa1, 0x2c, 0x1a, 0x86, 0x08, 0xa4, 0x77, 0x70, 0x8f, 0xa8, 0x2e, 0x29, 0x37, 0x6c, 0x44, - 0xe5, 0x48, 0xf5, 0x5b, 0xf0, 0xf0, 0xf6, 0xdb, 0x1e, 0x96, 0xc1, 0xac, 0x63, 0xab, 0x79, 0x73, - 0x0d, 0x10, 0x34, 0x9b, 0xb3, 0x46, 0xcb, 0x9c, 0x75, 0xec, 0xea, 0x16, 0x40, 0x93, 0xee, 0x6f, - 0xb8, 0x06, 0x00, 0x57, 0x94, 0x8e, 0x63, 0x73, 0xd5, 0xfb, 0xe7, 0x1a, 0xc5, 0xc1, 0xd5, 0x4a, - 0x4e, 0xb3, 0x8d, 0x16, 0x37, 0x73, 0x5a, 0x60, 0xd8, 0xbc, 0xf1, 0xf4, 0xf2, 0xff, 0xca, 0xcc, - 0xe5, 0xa0, 0x92, 0x7a, 0x37, 0xa8, 0xa4, 0xde, 0x0f, 0x2a, 0xa9, 0xff, 0x06, 0x95, 0xd4, 0x5f, - 0xd7, 0x95, 0x99, 0x77, 0xd7, 0x95, 0x99, 0xf7, 0xd7, 0x95, 0x99, 0x83, 0x7c, 0xec, 0xff, 0x95, - 0x0f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x05, 0xbc, 0xaf, 0x7b, 0xbe, 0x0d, 0x00, 0x00, +var fileDescriptor_processors_29d19df42cdc68ae = []byte{ + // 1245 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x97, 0xdf, 0x72, 0x13, 0xb7, + 0x17, 0xc7, 0xe3, 0xc4, 0x24, 0xb6, 0x1c, 0x13, 0xa3, 0x00, 0x3f, 0xfd, 0xdc, 0xd6, 0x49, 0x5d, + 0x68, 0x5d, 0x0a, 0xa6, 0x65, 0x3a, 0xbd, 0x60, 0xda, 0x69, 0xb1, 0x5d, 0x26, 0x9b, 0xd2, 0x40, + 0xd7, 0x29, 0xcc, 0x70, 0xe3, 0x2a, 0xbb, 0x8a, 0xb3, 0xb0, 0x5e, 0x2d, 0x92, 0x4c, 0x80, 0xbe, + 0x44, 0x1f, 0xa1, 0xef, 0xd1, 0x17, 0xc8, 0x55, 0x87, 0x4b, 0xae, 0x98, 0xd6, 0xbc, 0x48, 0x47, + 0x47, 0xeb, 0xdd, 0xb5, 0x13, 0xef, 0x72, 0x93, 0x71, 0xe4, 0xef, 0xf7, 0xa3, 0xa3, 0x73, 0x8e, + 0xfe, 0x18, 0x6d, 0xcb, 0x67, 0xfe, 0x4d, 0xf6, 0x82, 0x39, 0x5e, 0x70, 0x28, 0x68, 0x78, 0x70, + 0x33, 0x14, 0xdc, 0x61, 0x52, 0x72, 0x21, 0xdb, 0xa1, 0xe0, 0x8a, 0x63, 0xe2, 0x70, 0xe7, 0xa9, + 0xe0, 0xd4, 0x39, 0x6a, 0xcb, 0x67, 0x7e, 0xdb, 0xf5, 0xa4, 0x92, 0xcf, 0x7c, 0x31, 0x0e, 0xea, + 0xf5, 0x79, 0xaf, 0x4b, 0x15, 0x35, 0xae, 0xfa, 0xd5, 0xc5, 0xdc, 0xc1, 0x01, 0x95, 0x2c, 0x92, + 0x5d, 0xc9, 0x90, 0xe9, 0xd9, 0x8c, 0xaa, 0x95, 0x05, 0x1b, 0xfb, 0x4f, 0x07, 0x1e, 0x8f, 0x94, + 0xd7, 0x33, 0x94, 0xce, 0x11, 0x0d, 0x86, 0xec, 0x90, 0x31, 0x57, 0xbe, 0x87, 0x5a, 0xd1, 0x03, + 0x9f, 0x0d, 0xa4, 0xa2, 0x6a, 0xaa, 0xbe, 0xa4, 0xd5, 0xea, 0x65, 0xc8, 0xa4, 0xf9, 0x1b, 0x0d, + 0x5f, 0x1c, 0xf2, 0x21, 0x87, 0x8f, 0x37, 0xf5, 0x27, 0x33, 0xda, 0xfc, 0x7b, 0x05, 0x55, 0x1f, + 0x4c, 0x69, 0xfd, 0x90, 0x39, 0xb8, 0x8b, 0xce, 0x79, 0x41, 0x38, 0x56, 0xa4, 0xb0, 0xbd, 0xd2, + 0xaa, 0xdc, 0xfa, 0xac, 0xbd, 0x28, 0xaf, 0x6d, 0x4b, 0xcb, 0xfa, 0x2f, 0x03, 0x47, 0xfb, 0x3a, + 0xc5, 0x93, 0xb7, 0x5b, 0x4b, 0xb6, 0xf1, 0xe2, 0xbb, 0xa8, 0xe8, 0x70, 0xc1, 0xc8, 0xf2, 0x76, + 0xa1, 0x55, 0xb9, 0x75, 0x7d, 0x31, 0x23, 0x9e, 0xbb, 0xcb, 0x05, 0xfb, 0x35, 0xf0, 0x78, 0x10, + 0x81, 0xc0, 0x8f, 0x77, 0xd0, 0x2a, 0x1f, 0x2b, 0x1d, 0xcd, 0x0a, 0x44, 0x73, 0x6d, 0x31, 0xe9, + 0x3e, 0xe8, 0x6c, 0x3e, 0x56, 0x4c, 0xa4, 0x02, 0x8a, 0xfc, 0xb8, 0x8b, 0x8a, 0x21, 0x97, 0x8a, + 0x14, 0x21, 0xa2, 0xcf, 0x33, 0x22, 0xe2, 0x52, 0x45, 0x51, 0xa5, 0x30, 0x60, 0xc6, 0xd7, 0x50, + 0x49, 0x2a, 0x3a, 0x64, 0x03, 0xcf, 0x25, 0xe7, 0xb6, 0x0b, 0xad, 0x73, 0x9d, 0x0d, 0xfd, 0xed, + 0xe4, 0xed, 0xd6, 0x5a, 0x5f, 0x8f, 0x5b, 0x3d, 0x7b, 0x0d, 0x04, 0x96, 0x8b, 0xbf, 0x41, 0xeb, + 0x71, 0x99, 0xb4, 0x7e, 0x15, 0xf4, 0x9b, 0x91, 0xbe, 0x12, 0x2f, 0xdc, 0xea, 0xd9, 0x95, 0x58, + 0x68, 0xb9, 0xf8, 0x3b, 0xb4, 0x2e, 0x98, 0x1c, 0xfb, 0x6a, 0x00, 0xd5, 0x23, 0x6b, 0xb0, 0xf0, + 0xfa, 0x5c, 0xc0, 0x92, 0x8d, 0xda, 0xa6, 0xba, 0xfb, 0x76, 0xc5, 0xe8, 0xf7, 0xf5, 0xbf, 0xcd, + 0xbf, 0x36, 0x11, 0x3e, 0x9d, 0x54, 0x7c, 0x1b, 0x15, 0x03, 0xce, 0x43, 0x52, 0x80, 0xe5, 0x7f, + 0xba, 0x78, 0xf9, 0x7b, 0x9c, 0x87, 0xda, 0xa6, 0xd7, 0x6e, 0x83, 0x07, 0xff, 0x84, 0x2a, 0xd0, + 0x65, 0x36, 0xa3, 0x2e, 0x13, 0x51, 0x4d, 0x33, 0x32, 0xb8, 0x9f, 0x88, 0x81, 0x92, 0x76, 0xe3, + 0x1d, 0x84, 0x9e, 0x70, 0x2f, 0x88, 0x58, 0x2b, 0xc0, 0x6a, 0x2d, 0x66, 0xed, 0xc6, 0x5a, 0x40, + 0xa5, 0xbc, 0xf8, 0x5b, 0xb4, 0x2a, 0xb9, 0x50, 0x4c, 0x44, 0x35, 0xbd, 0xb2, 0x98, 0xd2, 0x07, + 0x1d, 0x10, 0x22, 0x8f, 0x8e, 0x83, 0x0e, 0x87, 0x82, 0x0d, 0xa9, 0xe2, 0x02, 0x8a, 0x99, 0x19, + 0xc7, 0x9d, 0x58, 0x6b, 0xe2, 0x48, 0xbc, 0xb8, 0x83, 0x4a, 0x5a, 0xe8, 0x05, 0x8e, 0x22, 0x6b, + 0x79, 0xe9, 0xed, 0x45, 0x4a, 0xa0, 0xc4, 0x3e, 0x9d, 0xe2, 0x11, 0x13, 0x43, 0xa6, 0x97, 0xcb, + 0x04, 0x29, 0xe5, 0xa5, 0xf8, 0xe7, 0x44, 0x6c, 0x52, 0x9c, 0x72, 0xeb, 0xa5, 0x1d, 0x51, 0x79, + 0x14, 0xb1, 0xca, 0x79, 0x4b, 0xdb, 0x89, 0xb5, 0x66, 0x69, 0x89, 0x17, 0xff, 0x80, 0x56, 0x9f, + 0x53, 0x7f, 0xcc, 0x24, 0x41, 0x79, 0x94, 0x87, 0xa0, 0x8b, 0x3b, 0x27, 0xf2, 0xe9, 0x58, 0x0e, + 0xa8, 0xf3, 0xf4, 0xd0, 0xf3, 0x7d, 0x26, 0x48, 0x25, 0x8f, 0xd2, 0x89, 0xb5, 0x26, 0x96, 0xc4, + 0x8b, 0xef, 0x21, 0x24, 0x18, 0x75, 0xad, 0x51, 0xc8, 0x85, 0x22, 0xd5, 0xbc, 0x83, 0xc5, 0x8e, + 0xb5, 0x3d, 0xaa, 0xa8, 0xa1, 0x25, 0x7e, 0xfc, 0x3d, 0x5a, 0xeb, 0xd3, 0x51, 0xa8, 0x83, 0xda, + 0x00, 0xd4, 0xd5, 0x8c, 0xee, 0x31, 0x42, 0x60, 0x4c, 0x5d, 0xf8, 0x31, 0xaa, 0x99, 0x8f, 0x49, + 0x67, 0x90, 0x1a, 0x90, 0xda, 0x79, 0xa4, 0xb9, 0x5e, 0x3a, 0xc5, 0xc1, 0xbf, 0x21, 0x3c, 0x62, + 0x8a, 0xea, 0x6b, 0x6a, 0x9f, 0x49, 0xd5, 0x67, 0x81, 0xde, 0x2b, 0x18, 0xe8, 0x5f, 0x66, 0x35, + 0xc5, 0xbc, 0x07, 0xf8, 0x67, 0xb0, 0xf0, 0x21, 0xba, 0x98, 0x1e, 0xb5, 0x99, 0xc3, 0xbc, 0xe7, + 0x4c, 0x90, 0x4d, 0x98, 0xe3, 0xd6, 0xfb, 0xcd, 0x31, 0x75, 0xc1, 0x2c, 0x67, 0xf2, 0xf0, 0x8f, + 0xa8, 0xdc, 0xed, 0x3f, 0x7c, 0x24, 0x3c, 0xbd, 0x4d, 0x2f, 0x02, 0x3c, 0xe3, 0x42, 0x89, 0xa5, + 0x40, 0x4c, 0x9c, 0x78, 0x0f, 0xad, 0xbf, 0xf2, 0x86, 0xaf, 0xe8, 0x30, 0xea, 0xe9, 0x4b, 0x40, + 0xca, 0xb8, 0x0c, 0x1e, 0xa7, 0xd4, 0x00, 0x9b, 0xf1, 0xeb, 0xae, 0x0c, 0x05, 0x7f, 0xc2, 0x1c, + 0xd5, 0x67, 0x8a, 0x5c, 0xce, 0xeb, 0xca, 0x07, 0xb1, 0xd6, 0xf4, 0x51, 0xe2, 0xd5, 0x9b, 0xff, + 0xd8, 0x0b, 0x5c, 0x7e, 0xcc, 0x04, 0xf9, 0x5f, 0xde, 0xe6, 0x7f, 0x14, 0x29, 0xcd, 0xe6, 0x9f, + 0xfa, 0xf0, 0x2f, 0xa8, 0xea, 0x73, 0x87, 0xfa, 0x0f, 0x7c, 0x1a, 0xec, 0x71, 0x97, 0x11, 0x02, + 0xa0, 0x2f, 0x16, 0x83, 0xee, 0xa5, 0xe5, 0x40, 0x9b, 0x25, 0xe8, 0xee, 0x34, 0xcf, 0x88, 0x54, + 0x77, 0xfe, 0x3f, 0xaf, 0x3b, 0xbb, 0x73, 0x0e, 0xd3, 0x9d, 0xf3, 0x1c, 0xbc, 0x8f, 0xce, 0x9b, + 0xb1, 0xbb, 0x82, 0x07, 0xca, 0x63, 0x82, 0xd4, 0xf3, 0x36, 0x63, 0x77, 0x46, 0x0f, 0xdc, 0x39, + 0x86, 0x2e, 0x09, 0x17, 0xae, 0x17, 0x50, 0xdf, 0x53, 0x2f, 0xc9, 0x07, 0x79, 0x25, 0xb9, 0x1f, + 0x6b, 0x4d, 0x49, 0x12, 0xaf, 0x4e, 0xa7, 0x7e, 0x6c, 0xd9, 0xfc, 0x38, 0xea, 0xbb, 0x0f, 0xf3, + 0xd2, 0xd9, 0x49, 0xcb, 0x4d, 0x3a, 0x67, 0x08, 0x3a, 0x9d, 0x5e, 0xf0, 0x9c, 0x09, 0xc5, 0xdc, + 0xbb, 0x9e, 0xaf, 0x98, 0x60, 0x82, 0x7c, 0x94, 0x97, 0x4e, 0x6b, 0xce, 0x61, 0xd2, 0x39, 0xcf, + 0xd1, 0xe9, 0x9c, 0x8e, 0x45, 0xdd, 0xdd, 0xc8, 0x4b, 0xa7, 0x35, 0xa3, 0x37, 0xe9, 0x9c, 0x65, + 0x4c, 0xcf, 0xdd, 0x71, 0xa8, 0x4f, 0x3f, 0xb2, 0xf5, 0x3e, 0xe7, 0xae, 0xd1, 0x26, 0xe7, 0xae, + 0xf9, 0x1f, 0x3f, 0x42, 0x1b, 0x32, 0xf4, 0x3d, 0x75, 0x27, 0x70, 0xfb, 0x0e, 0x55, 0x3a, 0xa1, + 0xdb, 0x80, 0xbb, 0x91, 0x71, 0xce, 0xcd, 0x1a, 0x80, 0x39, 0x4f, 0xd1, 0x77, 0x9e, 0x60, 0x52, + 0x71, 0xc1, 0x20, 0xc6, 0x8f, 0xf3, 0xee, 0x3c, 0x3b, 0x11, 0x9b, 0x3b, 0x2f, 0xe5, 0xd6, 0xfb, + 0xf0, 0x70, 0x5a, 0x99, 0x66, 0xde, 0x3e, 0x9c, 0xa9, 0x48, 0xec, 0xc3, 0x0e, 0xda, 0x94, 0x4a, + 0x30, 0x3a, 0xb2, 0x82, 0x21, 0x93, 0xca, 0xe3, 0x01, 0x04, 0xf6, 0x09, 0xe0, 0xbe, 0xca, 0x58, + 0xed, 0x69, 0x13, 0x90, 0xcf, 0xa2, 0xdd, 0x2e, 0x9e, 0xfc, 0xb9, 0x55, 0xd8, 0x2d, 0x96, 0xce, + 0xd7, 0x36, 0x76, 0x8b, 0xa5, 0x0b, 0x35, 0xbc, 0x5b, 0x2c, 0xad, 0xd6, 0xd6, 0x76, 0x8b, 0xa5, + 0xf5, 0x5a, 0xb5, 0x79, 0x1e, 0xad, 0xa7, 0x1f, 0x60, 0xcd, 0xdf, 0xd1, 0x85, 0x53, 0x7b, 0x1d, + 0xb7, 0xd0, 0xba, 0xcd, 0x8f, 0xfb, 0x7c, 0x2c, 0x1c, 0x66, 0xb9, 0x2f, 0xe0, 0x4d, 0x57, 0x8d, + 0xde, 0xa9, 0x33, 0xdf, 0xe0, 0x26, 0x2a, 0xef, 0x8d, 0x47, 0xf0, 0x4e, 0x97, 0xf0, 0x6e, 0x9b, + 0xca, 0x92, 0x61, 0x4c, 0x50, 0x71, 0x8f, 0x8e, 0x18, 0x3c, 0xc5, 0xca, 0xd3, 0xd7, 0xae, 0x1e, + 0x69, 0x7e, 0x8d, 0x2e, 0x9f, 0x7d, 0xa5, 0xe0, 0x3a, 0x5a, 0xf6, 0x5c, 0x98, 0xb7, 0xdc, 0x41, + 0xd1, 0x8b, 0x76, 0xd9, 0xea, 0xd9, 0xcb, 0x9e, 0xdb, 0xdc, 0x41, 0x64, 0xd1, 0x25, 0x81, 0xaf, + 0x23, 0x24, 0x81, 0x32, 0xf0, 0x5c, 0x09, 0x3f, 0x30, 0xca, 0x9d, 0xea, 0xe4, 0xed, 0x56, 0xd9, + 0xb0, 0xad, 0x9e, 0xb4, 0xcb, 0x46, 0x60, 0xb9, 0xb2, 0x73, 0xe3, 0xe4, 0xdf, 0xc6, 0xd2, 0xc9, + 0xa4, 0x51, 0x78, 0x3d, 0x69, 0x14, 0xde, 0x4c, 0x1a, 0x85, 0x7f, 0x26, 0x8d, 0xc2, 0x1f, 0xef, + 0x1a, 0x4b, 0xaf, 0xdf, 0x35, 0x96, 0xde, 0xbc, 0x6b, 0x2c, 0x3d, 0xae, 0xa4, 0x7e, 0x14, 0xfd, + 0x17, 0x00, 0x00, 0xff, 0xff, 0x81, 0xe1, 0xe4, 0x3b, 0x23, 0x0e, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 286bdaedaf1e..222ef2e97ea6 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -117,6 +117,7 @@ message ProcessorCoreUnion { optional SplitAndScatterSpec splitAndScatter = 32; optional RestoreDataSpec restoreData = 33; optional FiltererSpec filterer = 34; + optional StreamIngestionDataSpec streamIngestionData = 35; reserved 6, 12; } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index 7dfb4741fdbc..03097fe0f52f 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -18,6 +18,7 @@ import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" +import github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import encoding_binary "encoding/binary" @@ -71,7 +72,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0} } type BackfillerSpec_Type int32 @@ -110,7 +111,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -142,7 +143,7 @@ func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} func (*BackfillerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -182,7 +183,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} func (*JobProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{1} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +246,7 @@ func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{2} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -283,7 +284,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec_ImportTable) ProtoMessage() {} func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{2, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,6 +309,39 @@ func (m *ReadImportDataSpec_ImportTable) XXX_DiscardUnknown() { var xxx_messageInfo_ReadImportDataSpec_ImportTable proto.InternalMessageInfo +type StreamIngestionDataSpec struct { + PartitionAddress []github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.PartitionAddress `protobuf:"bytes,1,rep,name=partition_address,json=partitionAddress,customtype=github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.PartitionAddress" json:"partition_address"` +} + +func (m *StreamIngestionDataSpec) Reset() { *m = StreamIngestionDataSpec{} } +func (m *StreamIngestionDataSpec) String() string { return proto.CompactTextString(m) } +func (*StreamIngestionDataSpec) ProtoMessage() {} +func (*StreamIngestionDataSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{3} +} +func (m *StreamIngestionDataSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionDataSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StreamIngestionDataSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionDataSpec.Merge(dst, src) +} +func (m *StreamIngestionDataSpec) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionDataSpec) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionDataSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionDataSpec proto.InternalMessageInfo + type BackupDataSpec struct { Spans []roachpb.Span `protobuf:"bytes,1,rep,name=spans" json:"spans"` IntroducedSpans []roachpb.Span `protobuf:"bytes,2,rep,name=introduced_spans,json=introducedSpans" json:"introduced_spans"` @@ -329,7 +363,7 @@ func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} } func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) } func (*BackupDataSpec) ProtoMessage() {} func (*BackupDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{3} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{4} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +402,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} } func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) } func (*RestoreSpanEntry) ProtoMessage() {} func (*RestoreSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{4} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{5} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +440,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} } func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) } func (*RestoreDataSpec) ProtoMessage() {} func (*RestoreDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{5} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{6} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -440,7 +474,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} } func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec) ProtoMessage() {} func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{6} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{7} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +507,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {} func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{6, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{7, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -520,7 +554,7 @@ func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} func (*CSVWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{7} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{8} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -555,7 +589,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} } func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) } func (*BulkRowWriterSpec) ProtoMessage() {} func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{8} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{9} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -588,6 +622,7 @@ func init() { proto.RegisterMapType((map[string]*ReadImportDataSpec_ImportTable)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.TablesEntry") proto.RegisterMapType((map[int32]string)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.UriEntry") proto.RegisterType((*ReadImportDataSpec_ImportTable)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.ImportTable") + proto.RegisterType((*StreamIngestionDataSpec)(nil), "cockroach.sql.distsqlrun.StreamIngestionDataSpec") proto.RegisterType((*BackupDataSpec)(nil), "cockroach.sql.distsqlrun.BackupDataSpec") proto.RegisterMapType((map[uint64]bool)(nil), "cockroach.sql.distsqlrun.BackupDataSpec.PkIdsEntry") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.distsqlrun.BackupDataSpec.UrisByLocalityKvEntry") @@ -856,6 +891,39 @@ func (m *ReadImportDataSpec_ImportTable) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionDataSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StreamIngestionDataSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.PartitionAddress) > 0 { + for _, s := range m.PartitionAddress { + dAtA[i] = 0xa + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + return i, nil +} + func (m *BackupDataSpec) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1348,6 +1416,21 @@ func (m *ReadImportDataSpec_ImportTable) Size() (n int) { return n } +func (m *StreamIngestionDataSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.PartitionAddress) > 0 { + for _, s := range m.PartitionAddress { + l = len(s) + n += 1 + l + sovProcessorsBulkIo(uint64(l)) + } + } + return n +} + func (m *BackupDataSpec) Size() (n int) { if m == nil { return 0 @@ -2475,6 +2558,85 @@ func (m *ReadImportDataSpec_ImportTable) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionDataSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StreamIngestionDataSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionDataSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PartitionAddress = append(m.PartitionAddress, github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.PartitionAddress(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessorsBulkIo(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *BackupDataSpec) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -3929,114 +4091,119 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_bf46a169ea516277) -} - -var fileDescriptor_processors_bulk_io_bf46a169ea516277 = []byte{ - // 1673 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, - 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x24, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26, - 0x72, 0x59, 0x17, 0x26, 0x11, 0xbb, 0x2d, 0x8c, 0xb6, 0x89, 0x6b, 0x52, 0x96, 0x43, 0x39, 0xb1, - 0xd5, 0xa5, 0xa5, 0x00, 0x41, 0x8b, 0xc5, 0x72, 0x77, 0x44, 0x8d, 0xb9, 0xdc, 0x59, 0xcd, 0xcc, - 0xca, 0xa6, 0x4f, 0x05, 0x7a, 0xea, 0xad, 0x1f, 0xa1, 0xdf, 0xa0, 0xfd, 0x0e, 0xbd, 0xf8, 0x98, - 0x63, 0x4e, 0x46, 0x2b, 0x7f, 0x8b, 0x9e, 0x8a, 0xf9, 0xb3, 0xd4, 0x4a, 0xa6, 0x64, 0x29, 0x86, - 0x2f, 0xf4, 0x7a, 0xde, 0xfc, 0x7e, 0xf3, 0xe6, 0xcd, 0x7b, 0xbf, 0x37, 0x23, 0x68, 0xf2, 0xc3, - 0xa8, 0x8d, 0x5f, 0xe0, 0x80, 0xc4, 0xfb, 0xcc, 0x4f, 0x06, 0xed, 0x84, 0xd1, 0x00, 0x73, 0x4e, - 0x19, 0xf7, 0x06, 0x69, 0x34, 0xf2, 0x08, 0x6d, 0x25, 0x8c, 0x0a, 0x8a, 0x9c, 0x80, 0x06, 0x23, - 0x46, 0xfd, 0xe0, 0xa0, 0xc5, 0x0f, 0xa3, 0x56, 0x48, 0xb8, 0xe0, 0x87, 0x11, 0x4b, 0xe3, 0x95, - 0x4f, 0x9e, 0xd1, 0x01, 0x6f, 0xcb, 0x9f, 0x64, 0xa0, 0xfe, 0xd1, 0x88, 0x15, 0x47, 0xcd, 0x4e, - 0x06, 0x6d, 0x42, 0x6f, 0xed, 0x53, 0x36, 0xf6, 0x45, 0x66, 0xf9, 0x54, 0xae, 0x1a, 0xf8, 0xc2, - 0x8f, 0xe8, 0xb0, 0x1d, 0x62, 0x1e, 0x24, 0x83, 0x36, 0x17, 0x2c, 0x0d, 0x44, 0xca, 0x70, 0x68, - 0x26, 0x6d, 0x5c, 0xe4, 0x9a, 0xcf, 0x71, 0xb6, 0x4a, 0x2a, 0x48, 0xd4, 0x3e, 0x88, 0x82, 0xb6, - 0x20, 0x63, 0xcc, 0x85, 0x3f, 0x4e, 0x8c, 0xe5, 0xfa, 0x90, 0x0e, 0xa9, 0xfa, 0x6c, 0xcb, 0x2f, - 0x33, 0x8a, 0x32, 0xaf, 0x42, 0x5f, 0xf8, 0x66, 0x6c, 0x39, 0x1b, 0xf3, 0x13, 0xa2, 0x87, 0x1a, - 0xff, 0x2c, 0xc2, 0x62, 0xc7, 0x0f, 0x46, 0xfb, 0x24, 0x8a, 0x30, 0xeb, 0x27, 0x38, 0x40, 0x0f, - 0xc1, 0x16, 0x93, 0x04, 0x3b, 0xd6, 0xba, 0xd5, 0x5c, 0xbc, 0x7d, 0xab, 0x75, 0x5e, 0x40, 0x5a, - 0xa7, 0x71, 0xad, 0xa7, 0x93, 0x04, 0x77, 0xec, 0x57, 0xaf, 0xd7, 0xe6, 0x5c, 0x45, 0x80, 0x3a, - 0x50, 0x12, 0xfe, 0x20, 0xc2, 0x4e, 0x61, 0xdd, 0x6a, 0xd6, 0x6e, 0xdf, 0x38, 0xc3, 0xc4, 0x0f, - 0x23, 0xb5, 0xbf, 0xa7, 0x72, 0xce, 0x26, 0xe6, 0x01, 0x23, 0x89, 0xa0, 0xcc, 0x50, 0x68, 0x28, - 0x7a, 0x00, 0x25, 0x9e, 0xf8, 0x31, 0x77, 0x8a, 0xeb, 0xc5, 0x66, 0xed, 0xf6, 0x2f, 0xce, 0xf7, - 0x46, 0xd1, 0xb8, 0xd8, 0x0f, 0xa5, 0x3b, 0x7e, 0x9c, 0xd1, 0x28, 0x34, 0xfa, 0x1c, 0x2a, 0x61, - 0xca, 0x7c, 0x41, 0x68, 0xec, 0xd8, 0xeb, 0x56, 0xb3, 0xd8, 0xf9, 0x58, 0x9a, 0xff, 0xf7, 0x7a, - 0x6d, 0x41, 0x86, 0xb3, 0xb5, 0x69, 0x8c, 0xee, 0x74, 0x1a, 0xfa, 0x14, 0x20, 0x38, 0x48, 0xe3, - 0x91, 0xc7, 0xc9, 0x4b, 0xec, 0x94, 0x14, 0x48, 0x73, 0x56, 0xd5, 0x78, 0x9f, 0xbc, 0xc4, 0xe8, - 0x1e, 0x54, 0x18, 0xf6, 0xc3, 0xfb, 0xfc, 0xc9, 0xbe, 0x33, 0xaf, 0x76, 0xf9, 0xb3, 0x9c, 0x87, - 0xf2, 0xc8, 0x5a, 0x07, 0x51, 0xd0, 0x7a, 0x9a, 0x1d, 0x99, 0x61, 0x98, 0x82, 0x1a, 0x37, 0xc1, - 0x96, 0x71, 0x43, 0x35, 0x98, 0xef, 0xc5, 0x47, 0x7e, 0x44, 0xc2, 0xfa, 0x1c, 0x02, 0x28, 0x77, - 0x69, 0x94, 0x8e, 0xe3, 0xba, 0x85, 0xaa, 0x50, 0xea, 0xc5, 0x21, 0x7e, 0x51, 0x2f, 0x6c, 0xdb, - 0x95, 0x72, 0x7d, 0xbe, 0xf1, 0x1c, 0x6a, 0xdb, 0x74, 0xb0, 0xc3, 0xe8, 0x90, 0x61, 0xce, 0xd1, - 0x67, 0x50, 0x7e, 0x46, 0x07, 0x1e, 0x09, 0xd5, 0x79, 0x15, 0x3b, 0x0b, 0x72, 0x81, 0xe3, 0xd7, - 0x6b, 0xa5, 0x6d, 0x3a, 0xe8, 0x6d, 0xba, 0xa5, 0x67, 0x74, 0xd0, 0x0b, 0x51, 0x13, 0xae, 0x05, - 0x34, 0x16, 0x8c, 0x0c, 0x52, 0x15, 0x03, 0x79, 0x22, 0x05, 0xe3, 0xcc, 0x29, 0x0b, 0x72, 0xc0, - 0xe6, 0x11, 0x15, 0x4e, 0x71, 0xdd, 0x6a, 0x96, 0xb2, 0xe3, 0x94, 0x23, 0x8d, 0x57, 0x15, 0x40, - 0x32, 0xbe, 0xbd, 0x71, 0x42, 0x99, 0xd8, 0xf4, 0x85, 0xaf, 0xd2, 0x65, 0x03, 0x6a, 0xdc, 0x1f, - 0x27, 0x11, 0xd6, 0x81, 0x2a, 0xe4, 0x70, 0xa0, 0x0d, 0x2a, 0x52, 0x0f, 0xa1, 0x92, 0x18, 0x9f, - 0x9d, 0xb2, 0x8a, 0xd4, 0xc6, 0xf9, 0x67, 0x99, 0xdb, 0x60, 0x16, 0xb1, 0x0c, 0x8c, 0x1e, 0x42, - 0x31, 0x65, 0xc4, 0x99, 0x57, 0xf9, 0xf0, 0xeb, 0xf3, 0x39, 0xde, 0x76, 0xb5, 0xb5, 0xcb, 0xc8, - 0x83, 0x58, 0xb0, 0x89, 0x2b, 0x19, 0xd0, 0x17, 0x50, 0xd6, 0xe5, 0xea, 0x54, 0x94, 0x3f, 0x6b, - 0x39, 0x2e, 0x53, 0x28, 0xad, 0xde, 0x93, 0x2d, 0x12, 0xe1, 0x2d, 0x35, 0xcd, 0x78, 0x62, 0x40, - 0x68, 0x0f, 0xca, 0x2a, 0x45, 0xb9, 0x53, 0x55, 0xae, 0xdc, 0xbd, 0x92, 0x2b, 0x2a, 0x5b, 0xb9, - 0xf2, 0x46, 0xf1, 0x5a, 0xae, 0x61, 0x43, 0xf7, 0xe0, 0xa7, 0x7c, 0x44, 0x12, 0x6f, 0x4c, 0x38, - 0x27, 0xf1, 0xd0, 0xdb, 0xa7, 0x0c, 0x93, 0x61, 0xec, 0x8d, 0xf0, 0x84, 0x3b, 0xb0, 0x6e, 0x35, - 0x2b, 0xc6, 0x91, 0x4f, 0xe4, 0xb4, 0x6f, 0xf4, 0xac, 0x2d, 0x3d, 0xe9, 0x11, 0x9e, 0x70, 0x74, - 0x13, 0x16, 0x9e, 0xfb, 0x51, 0x24, 0xf3, 0xfa, 0xb1, 0x1f, 0x53, 0xee, 0xd4, 0x72, 0xb9, 0x7b, - 0xda, 0x84, 0x6e, 0xc3, 0x32, 0x53, 0x25, 0xb3, 0xe3, 0x33, 0x3f, 0x8a, 0x70, 0x44, 0xf8, 0xd8, - 0x59, 0xc8, 0x1d, 0xe1, 0xdb, 0x66, 0xf4, 0x1d, 0x00, 0xc3, 0x3c, 0x1d, 0x63, 0x2f, 0xa1, 0xdc, - 0x59, 0x54, 0x9b, 0xff, 0xdd, 0x95, 0x36, 0xef, 0x2a, 0xf8, 0x0e, 0xd5, 0xfb, 0x77, 0xab, 0x2c, - 0xfb, 0x3f, 0xc2, 0x00, 0x29, 0xc7, 0xcc, 0x53, 0xe2, 0xe4, 0x2c, 0xad, 0x5b, 0xcd, 0x6a, 0x67, - 0xcb, 0x54, 0xea, 0x97, 0x43, 0x22, 0x0e, 0xd2, 0x41, 0x2b, 0xa0, 0xe3, 0xf6, 0x74, 0xb5, 0x70, - 0x70, 0xf2, 0xdd, 0x4e, 0x46, 0xc3, 0x36, 0xc7, 0x41, 0xca, 0x88, 0x98, 0xb4, 0xfa, 0x7f, 0xfc, - 0x7a, 0x97, 0x63, 0x16, 0xfb, 0x63, 0xbc, 0x23, 0xd9, 0xdc, 0xaa, 0x64, 0x56, 0x9f, 0x2b, 0x29, - 0xd4, 0xb4, 0x4b, 0xea, 0x18, 0xd0, 0x1f, 0xc0, 0x96, 0xea, 0xac, 0x2a, 0xe8, 0x6a, 0x3a, 0x65, - 0xb9, 0x0a, 0x89, 0x3e, 0x03, 0x10, 0x3e, 0x1b, 0x62, 0xd1, 0xa5, 0x11, 0x77, 0x0a, 0xeb, 0xc5, - 0x66, 0xd5, 0xd8, 0x73, 0xe3, 0x2b, 0x1c, 0x6a, 0xb9, 0x73, 0x47, 0x75, 0x28, 0x8e, 0xf0, 0x44, - 0xad, 0x5a, 0x75, 0xe5, 0x27, 0x7a, 0x0c, 0xa5, 0x23, 0x3f, 0x4a, 0x33, 0xc5, 0xbc, 0x5a, 0x4a, - 0xe5, 0x76, 0xe4, 0x6a, 0x9a, 0xdf, 0x16, 0xee, 0x5a, 0x2b, 0xbf, 0x81, 0x4a, 0x96, 0xf7, 0xf9, - 0x15, 0x4b, 0x7a, 0xc5, 0xeb, 0xf9, 0x15, 0xab, 0x79, 0xdc, 0xef, 0x61, 0xf1, 0xf4, 0x39, 0xbd, - 0x0b, 0x5d, 0xcc, 0xa1, 0xb7, 0xed, 0x8a, 0xa5, 0x14, 0xab, 0x58, 0xb7, 0xb7, 0xed, 0x8a, 0x5d, - 0x2f, 0x6d, 0xdb, 0x95, 0x52, 0xbd, 0xbc, 0x6d, 0x57, 0xae, 0xd5, 0x17, 0x1a, 0xff, 0x9e, 0xd7, - 0x5d, 0x27, 0x4d, 0xa6, 0x32, 0x72, 0x27, 0x13, 0x7a, 0x4b, 0x25, 0xd4, 0x4f, 0x66, 0x14, 0xe3, - 0xdb, 0xb2, 0xfe, 0x15, 0xd4, 0x49, 0x2c, 0x18, 0x0d, 0xd3, 0x00, 0x87, 0x9e, 0xc6, 0x17, 0x2e, - 0x83, 0x5f, 0x3a, 0x81, 0xf5, 0x15, 0xd3, 0x1d, 0xa8, 0x85, 0x78, 0xdf, 0x4f, 0x23, 0xe1, 0x49, - 0x75, 0x29, 0xaa, 0xcc, 0x43, 0x46, 0x4b, 0x61, 0x53, 0x9b, 0x76, 0xdd, 0x9e, 0x0b, 0x66, 0xda, - 0x2e, 0x23, 0xe8, 0xaf, 0x16, 0x7c, 0x94, 0x32, 0xc2, 0xbd, 0xc1, 0xc4, 0x8b, 0x68, 0xe0, 0x47, - 0x44, 0x4c, 0xbc, 0xd1, 0x91, 0x63, 0x2b, 0x17, 0xbe, 0xbc, 0xb8, 0x73, 0x9e, 0xec, 0x5d, 0xea, - 0x12, 0xef, 0x4c, 0xbe, 0x36, 0x0c, 0x8f, 0x8e, 0xb4, 0x2c, 0x5c, 0x3f, 0x7e, 0xbd, 0x56, 0xdf, - 0x75, 0x7b, 0x79, 0xd3, 0x9e, 0x5b, 0x4f, 0xcf, 0x4c, 0x46, 0x2e, 0xd4, 0xc6, 0x47, 0x41, 0xe0, - 0xed, 0x93, 0x48, 0x60, 0xa6, 0x3a, 0xd5, 0xe2, 0xa9, 0x36, 0x94, 0xed, 0xff, 0x9b, 0xbd, 0x6e, - 0x77, 0x4b, 0x4d, 0x3a, 0xd9, 0xd9, 0xc9, 0x98, 0x0b, 0x92, 0x45, 0x7f, 0xa3, 0xaf, 0x00, 0x70, - 0x1c, 0xb0, 0x49, 0xa2, 0xba, 0x85, 0xd6, 0xeb, 0xe6, 0x0c, 0x4a, 0xa9, 0x8e, 0x0f, 0xa6, 0x13, - 0x9f, 0xa8, 0x5f, 0xee, 0xe6, 0xb0, 0xe8, 0x09, 0x2c, 0x0f, 0xd4, 0x6e, 0x3d, 0x2e, 0x7c, 0x26, - 0x3c, 0xa9, 0x3d, 0x57, 0x69, 0x95, 0x4b, 0x1a, 0xdd, 0x97, 0x60, 0x69, 0x43, 0x8f, 0xc0, 0x0c, - 0x79, 0x38, 0x0e, 0x35, 0x5d, 0xe5, 0xf2, 0x74, 0x0b, 0x1a, 0xfb, 0x20, 0x0e, 0x15, 0xd9, 0x2e, - 0x94, 0x93, 0x91, 0x47, 0xc2, 0x4c, 0xc4, 0xef, 0x5c, 0xfa, 0xcc, 0x76, 0x46, 0xbd, 0xd0, 0xe8, - 0x77, 0x55, 0xb6, 0xdb, 0x9d, 0x47, 0xbd, 0x4d, 0xee, 0x96, 0x12, 0x39, 0x7c, 0x46, 0xc6, 0xe0, - 0x43, 0xc9, 0x58, 0x17, 0x3e, 0x9e, 0x99, 0x3a, 0x33, 0x94, 0xe5, 0xfc, 0x3a, 0xbf, 0x0b, 0x70, - 0xb2, 0x97, 0x3c, 0xd2, 0x9e, 0x81, 0xac, 0xe4, 0x90, 0x8d, 0x7f, 0x59, 0x50, 0x77, 0x31, 0x17, - 0x94, 0x61, 0x59, 0x44, 0x9a, 0xe0, 0x73, 0xb0, 0x65, 0x1d, 0x1a, 0x2d, 0x7d, 0x47, 0x19, 0xaa, - 0xa9, 0xe8, 0x3e, 0x94, 0xf6, 0x89, 0x6c, 0xa4, 0xba, 0x74, 0x37, 0x66, 0xf5, 0x61, 0xa5, 0x6d, - 0x2e, 0x3e, 0x4c, 0x31, 0x17, 0x2a, 0xeb, 0x32, 0x21, 0x50, 0x48, 0x74, 0x03, 0x6a, 0xd9, 0x05, - 0xa1, 0x17, 0xbe, 0x50, 0xe5, 0x9b, 0x75, 0xbc, 0xbc, 0xa1, 0xf1, 0x97, 0x22, 0x2c, 0x19, 0x97, - 0xa7, 0xca, 0xb3, 0x05, 0xd7, 0x98, 0x1e, 0xd2, 0xd9, 0x64, 0x5d, 0x3e, 0x9b, 0x6a, 0x06, 0xa8, - 0x72, 0xe9, 0x74, 0xcd, 0x14, 0xde, 0xa3, 0x66, 0x7a, 0x50, 0x66, 0x58, 0xf5, 0x7b, 0x7d, 0xeb, - 0xfd, 0xe5, 0x3b, 0x23, 0x62, 0x2e, 0xbf, 0x23, 0x3c, 0xc9, 0x6e, 0x29, 0x9a, 0x40, 0xde, 0x52, - 0x4c, 0x82, 0x6b, 0x51, 0xfa, 0xd5, 0x45, 0x2d, 0xe5, 0x54, 0x5c, 0x2e, 0xcc, 0xf0, 0xf7, 0xc8, - 0x9a, 0x7f, 0x14, 0xe0, 0xa3, 0x7e, 0x12, 0x11, 0x71, 0x3f, 0x0e, 0xfb, 0x81, 0x2f, 0x84, 0x79, - 0x76, 0xfc, 0x19, 0xca, 0xea, 0x5e, 0x9d, 0x75, 0x80, 0x7b, 0xe7, 0x7b, 0x3a, 0x03, 0x9e, 0x79, - 0xaf, 0xfc, 0xe9, 0x4a, 0x9e, 0x2c, 0x10, 0x9a, 0x34, 0x17, 0xd3, 0xc2, 0x7b, 0xc6, 0x74, 0xc5, - 0x83, 0xe5, 0xb7, 0x56, 0x43, 0xdb, 0x30, 0x8f, 0xe5, 0x35, 0x1a, 0x67, 0xfe, 0xdf, 0x7c, 0x67, - 0xa4, 0xa7, 0x45, 0x63, 0xf8, 0x33, 0x82, 0xc6, 0xdf, 0x8a, 0xb0, 0xd0, 0xed, 0xef, 0x7d, 0xcb, - 0x48, 0x16, 0x9c, 0x1b, 0xb2, 0x3d, 0x71, 0x41, 0x62, 0xfd, 0x84, 0x51, 0x85, 0x9d, 0xe5, 0x60, - 0xce, 0x80, 0x7e, 0x0e, 0xd7, 0xa4, 0x52, 0x78, 0x89, 0x0a, 0x8c, 0xce, 0xc2, 0xe9, 0x44, 0xa5, - 0x21, 0xda, 0x80, 0xbe, 0x80, 0x79, 0xaa, 0x33, 0x4f, 0x15, 0x4b, 0x6d, 0x66, 0xc3, 0xe8, 0xf6, - 0xf7, 0x4c, 0x7a, 0x66, 0x1e, 0x1a, 0xcc, 0xc9, 0xe3, 0x88, 0xd1, 0xe7, 0xdc, 0xbc, 0xa8, 0xf2, - 0x8f, 0x23, 0x97, 0x3e, 0xe7, 0xe8, 0x4f, 0xb0, 0x1c, 0xd0, 0x71, 0x22, 0x6b, 0x8f, 0xd0, 0xd8, - 0x0b, 0x68, 0x88, 0x03, 0xd3, 0x9e, 0x2e, 0x78, 0xc7, 0xc9, 0xf2, 0xe8, 0x9e, 0xc0, 0x0c, 0x6d, - 0x3d, 0xc7, 0xd4, 0x95, 0x44, 0x67, 0x34, 0xb6, 0xfc, 0x81, 0x34, 0xb6, 0xf1, 0x2d, 0x2c, 0x77, - 0xd2, 0x48, 0x6e, 0x28, 0x77, 0x1c, 0xd3, 0x97, 0xad, 0xf5, 0xa3, 0x5f, 0xb6, 0x37, 0x37, 0x60, - 0xe9, 0xcc, 0x56, 0x51, 0x05, 0xec, 0xc7, 0x34, 0xc6, 0xf5, 0x39, 0xf9, 0xf5, 0xf0, 0x25, 0x49, - 0xea, 0x56, 0xe7, 0xd6, 0xab, 0xff, 0xae, 0xce, 0xbd, 0x3a, 0x5e, 0xb5, 0xbe, 0x3f, 0x5e, 0xb5, - 0x7e, 0x38, 0x5e, 0xb5, 0xfe, 0x73, 0xbc, 0x6a, 0xfd, 0xfd, 0xcd, 0xea, 0xdc, 0xf7, 0x6f, 0x56, - 0xe7, 0x7e, 0x78, 0xb3, 0x3a, 0xf7, 0x5d, 0x2d, 0xf7, 0xc7, 0x83, 0xff, 0x07, 0x00, 0x00, 0xff, - 0xff, 0x3b, 0xa7, 0xba, 0x6f, 0xe9, 0x10, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_a1c1a7c66115c204) +} + +var fileDescriptor_processors_bulk_io_a1c1a7c66115c204 = []byte{ + // 1747 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xb9, + 0x15, 0xf7, 0xe8, 0x9f, 0xa5, 0xa7, 0xd8, 0x1e, 0x73, 0xb3, 0xbb, 0x53, 0x03, 0xb5, 0x0d, 0xed, + 0x3a, 0x55, 0x53, 0x44, 0xc2, 0x26, 0x6d, 0x11, 0xb4, 0xdd, 0x4d, 0x63, 0x39, 0xce, 0xca, 0xd9, + 0x4d, 0xdc, 0x51, 0xec, 0x05, 0xb6, 0x2d, 0x06, 0xd4, 0x0c, 0x2d, 0x33, 0x1a, 0x0d, 0xc7, 0x24, + 0xc7, 0x89, 0x72, 0x2a, 0xd0, 0x4b, 0x7b, 0xeb, 0x47, 0xe8, 0xa1, 0xf7, 0xf6, 0x3b, 0xf4, 0x92, + 0xe3, 0x1e, 0x17, 0x3d, 0x18, 0xad, 0xf3, 0x2d, 0x7a, 0x2a, 0x48, 0xce, 0x48, 0x63, 0xc7, 0x76, + 0xec, 0x06, 0xb9, 0x38, 0x23, 0x3e, 0xfe, 0x7e, 0x7c, 0x7c, 0x7c, 0xef, 0xf7, 0xc8, 0x40, 0x53, + 0x1c, 0x84, 0x6d, 0xf2, 0x82, 0xf8, 0x34, 0xda, 0xe3, 0x38, 0xee, 0xb7, 0x63, 0xce, 0x7c, 0x22, + 0x04, 0xe3, 0xc2, 0xeb, 0x27, 0xe1, 0xd0, 0xa3, 0xac, 0x15, 0x73, 0x26, 0x19, 0x72, 0x7c, 0xe6, + 0x0f, 0x39, 0xc3, 0xfe, 0x7e, 0x4b, 0x1c, 0x84, 0xad, 0x80, 0x0a, 0x29, 0x0e, 0x42, 0x9e, 0x44, + 0x4b, 0x1f, 0x3d, 0x63, 0x7d, 0xd1, 0x56, 0x7f, 0xe2, 0xbe, 0xfe, 0xc7, 0x20, 0x96, 0x1c, 0x3d, + 0x3b, 0xee, 0xb7, 0x29, 0xbb, 0xb5, 0xc7, 0xf8, 0x08, 0xcb, 0xcc, 0xf2, 0x89, 0x5a, 0xd5, 0xc7, + 0x12, 0x87, 0x6c, 0xd0, 0x0e, 0x88, 0xf0, 0xe3, 0x7e, 0x5b, 0x48, 0x9e, 0xf8, 0x32, 0xe1, 0x24, + 0x48, 0x27, 0xad, 0x5d, 0xe4, 0x1a, 0x16, 0x24, 0x5b, 0x25, 0x91, 0x34, 0x6c, 0xef, 0x87, 0x7e, + 0x5b, 0xd2, 0x11, 0x11, 0x12, 0x8f, 0xe2, 0xd4, 0x72, 0x7d, 0xc0, 0x06, 0x4c, 0x7f, 0xb6, 0xd5, + 0x57, 0x3a, 0x8a, 0x32, 0xaf, 0x02, 0x2c, 0x71, 0x3a, 0xb6, 0x98, 0x8d, 0xe1, 0x98, 0x9a, 0xa1, + 0xc6, 0xdf, 0x8b, 0x30, 0xbf, 0x8e, 0xfd, 0xe1, 0x1e, 0x0d, 0x43, 0xc2, 0x7b, 0x31, 0xf1, 0xd1, + 0x43, 0x28, 0xc9, 0x71, 0x4c, 0x1c, 0x6b, 0xd5, 0x6a, 0xce, 0xdf, 0xbe, 0xd5, 0x3a, 0x2f, 0x20, + 0xad, 0x93, 0xb8, 0xd6, 0xd3, 0x71, 0x4c, 0xd6, 0x4b, 0xaf, 0x8e, 0x56, 0x66, 0x5c, 0x4d, 0x80, + 0xd6, 0xa1, 0x2c, 0x71, 0x3f, 0x24, 0x4e, 0x61, 0xd5, 0x6a, 0xd6, 0x6f, 0xdf, 0x38, 0xc5, 0x24, + 0x0e, 0x42, 0xbd, 0xbf, 0xa7, 0x6a, 0xce, 0x06, 0x11, 0x3e, 0xa7, 0xb1, 0x64, 0x3c, 0xa5, 0x30, + 0x50, 0xf4, 0x00, 0xca, 0x22, 0xc6, 0x91, 0x70, 0x8a, 0xab, 0xc5, 0x66, 0xfd, 0xf6, 0x8f, 0xcf, + 0xf7, 0x46, 0xd3, 0xb8, 0x04, 0x07, 0xca, 0x1d, 0x1c, 0x65, 0x34, 0x1a, 0x8d, 0x3e, 0x83, 0x6a, + 0x90, 0x70, 0x2c, 0x29, 0x8b, 0x9c, 0xd2, 0xaa, 0xd5, 0x2c, 0xae, 0x7f, 0xa8, 0xcc, 0xff, 0x3d, + 0x5a, 0x99, 0x53, 0xe1, 0x6c, 0x6d, 0xa4, 0x46, 0x77, 0x32, 0x0d, 0x7d, 0x02, 0xe0, 0xef, 0x27, + 0xd1, 0xd0, 0x13, 0xf4, 0x25, 0x71, 0xca, 0x1a, 0x64, 0x38, 0x6b, 0x7a, 0xbc, 0x47, 0x5f, 0x12, + 0x74, 0x0f, 0xaa, 0x9c, 0xe0, 0xe0, 0xbe, 0x78, 0xb2, 0xe7, 0xcc, 0xea, 0x5d, 0xfe, 0x30, 0xe7, + 0xa1, 0x3a, 0xb2, 0xd6, 0x7e, 0xe8, 0xb7, 0x9e, 0x66, 0x47, 0x96, 0x32, 0x4c, 0x40, 0x8d, 0x9b, + 0x50, 0x52, 0x71, 0x43, 0x75, 0x98, 0xed, 0x46, 0x87, 0x38, 0xa4, 0x81, 0x3d, 0x83, 0x00, 0x2a, + 0x1d, 0x16, 0x26, 0xa3, 0xc8, 0xb6, 0x50, 0x0d, 0xca, 0xdd, 0x28, 0x20, 0x2f, 0xec, 0xc2, 0x56, + 0xa9, 0x5a, 0xb1, 0x67, 0x1b, 0xcf, 0xa1, 0xbe, 0xc5, 0xfa, 0xdb, 0x9c, 0x0d, 0x38, 0x11, 0x02, + 0x7d, 0x0a, 0x95, 0x67, 0xac, 0xef, 0xd1, 0x40, 0x9f, 0x57, 0x71, 0x7d, 0x4e, 0x2d, 0x70, 0x7c, + 0xb4, 0x52, 0xde, 0x62, 0xfd, 0xee, 0x86, 0x5b, 0x7e, 0xc6, 0xfa, 0xdd, 0x00, 0x35, 0xe1, 0x9a, + 0xcf, 0x22, 0xc9, 0x69, 0x3f, 0xd1, 0x31, 0x50, 0x27, 0x52, 0x48, 0x9d, 0x39, 0x61, 0x41, 0x0e, + 0x94, 0x44, 0xc8, 0xa4, 0x53, 0x5c, 0xb5, 0x9a, 0xe5, 0xec, 0x38, 0xd5, 0x48, 0xe3, 0x55, 0x15, + 0x90, 0x8a, 0x6f, 0x77, 0x14, 0x33, 0x2e, 0x37, 0xb0, 0xc4, 0x3a, 0x5d, 0xd6, 0xa0, 0x2e, 0xf0, + 0x28, 0x0e, 0x89, 0x09, 0x54, 0x21, 0x87, 0x03, 0x63, 0xd0, 0x91, 0x7a, 0x08, 0xd5, 0x38, 0xf5, + 0xd9, 0xa9, 0xe8, 0x48, 0xad, 0x9d, 0x7f, 0x96, 0xb9, 0x0d, 0x66, 0x11, 0xcb, 0xc0, 0xe8, 0x21, + 0x14, 0x13, 0x4e, 0x9d, 0x59, 0x9d, 0x0f, 0x3f, 0x3b, 0x9f, 0xe3, 0x4d, 0x57, 0x5b, 0x3b, 0x9c, + 0x3e, 0x88, 0x24, 0x1f, 0xbb, 0x8a, 0x01, 0x7d, 0x0e, 0x15, 0x53, 0xae, 0x4e, 0x55, 0xfb, 0xb3, + 0x92, 0xe3, 0x4a, 0x0b, 0xa5, 0xd5, 0x7d, 0xb2, 0x49, 0x43, 0xb2, 0xa9, 0xa7, 0xa5, 0x9e, 0xa4, + 0x20, 0xb4, 0x0b, 0x15, 0x9d, 0xa2, 0xc2, 0xa9, 0x69, 0x57, 0xee, 0x5e, 0xc9, 0x15, 0x9d, 0xad, + 0x42, 0x7b, 0xa3, 0x79, 0x2d, 0x37, 0x65, 0x43, 0xf7, 0xe0, 0x07, 0x62, 0x48, 0x63, 0x6f, 0x44, + 0x85, 0xa0, 0xd1, 0xc0, 0xdb, 0x63, 0x9c, 0xd0, 0x41, 0xe4, 0x0d, 0xc9, 0x58, 0x38, 0xb0, 0x6a, + 0x35, 0xab, 0xa9, 0x23, 0x1f, 0xa9, 0x69, 0x5f, 0x9b, 0x59, 0x9b, 0x66, 0xd2, 0x23, 0x32, 0x16, + 0xe8, 0x26, 0xcc, 0x3d, 0xc7, 0x61, 0xa8, 0xf2, 0xfa, 0x31, 0x8e, 0x98, 0x70, 0xea, 0xb9, 0xdc, + 0x3d, 0x69, 0x42, 0xb7, 0x61, 0x91, 0xeb, 0x92, 0xd9, 0xc6, 0x1c, 0x87, 0x21, 0x09, 0xa9, 0x18, + 0x39, 0x73, 0xb9, 0x23, 0x7c, 0xd3, 0x8c, 0xbe, 0x05, 0xe0, 0x44, 0x24, 0x23, 0xe2, 0xc5, 0x4c, + 0x38, 0xf3, 0x7a, 0xf3, 0xbf, 0xbc, 0xd2, 0xe6, 0x5d, 0x0d, 0xdf, 0x66, 0x66, 0xff, 0x6e, 0x8d, + 0x67, 0xbf, 0x11, 0x01, 0x48, 0x04, 0xe1, 0x9e, 0x16, 0x27, 0x67, 0x61, 0xd5, 0x6a, 0xd6, 0xd6, + 0x37, 0xd3, 0x4a, 0xfd, 0x62, 0x40, 0xe5, 0x7e, 0xd2, 0x6f, 0xf9, 0x6c, 0xd4, 0x9e, 0xac, 0x16, + 0xf4, 0xa7, 0xdf, 0xed, 0x78, 0x38, 0x68, 0x0b, 0xe2, 0x27, 0x9c, 0xca, 0x71, 0xab, 0xf7, 0x9b, + 0xaf, 0x76, 0x04, 0xe1, 0x11, 0x1e, 0x91, 0x6d, 0xc5, 0xe6, 0xd6, 0x14, 0xb3, 0xfe, 0x5c, 0x4a, + 0xa0, 0x6e, 0x5c, 0xd2, 0xc7, 0x80, 0x7e, 0x0d, 0x25, 0xa5, 0xce, 0xba, 0x82, 0xae, 0xa6, 0x53, + 0x96, 0xab, 0x91, 0xe8, 0x53, 0x00, 0x89, 0xf9, 0x80, 0xc8, 0x0e, 0x0b, 0x85, 0x53, 0x58, 0x2d, + 0x36, 0x6b, 0xa9, 0x3d, 0x37, 0xbe, 0x24, 0xa0, 0x9e, 0x3b, 0x77, 0x64, 0x43, 0x71, 0x48, 0xc6, + 0x7a, 0xd5, 0x9a, 0xab, 0x3e, 0xd1, 0x63, 0x28, 0x1f, 0xe2, 0x30, 0xc9, 0x14, 0xf3, 0x6a, 0x29, + 0x95, 0xdb, 0x91, 0x6b, 0x68, 0x7e, 0x51, 0xb8, 0x6b, 0x2d, 0xfd, 0x1c, 0xaa, 0x59, 0xde, 0xe7, + 0x57, 0x2c, 0x9b, 0x15, 0xaf, 0xe7, 0x57, 0xac, 0xe5, 0x71, 0xbf, 0x82, 0xf9, 0x93, 0xe7, 0xf4, + 0x36, 0x74, 0x31, 0x87, 0xde, 0x2a, 0x55, 0x2d, 0xad, 0x58, 0x45, 0xbb, 0xb4, 0x55, 0xaa, 0x96, + 0xec, 0xf2, 0x56, 0xa9, 0x5a, 0xb6, 0x2b, 0x5b, 0xa5, 0xea, 0x35, 0x7b, 0xae, 0xf1, 0x37, 0x0b, + 0x3e, 0xee, 0x49, 0x4e, 0xf0, 0xa8, 0x1b, 0x0d, 0x88, 0x50, 0xc2, 0x33, 0xd1, 0x93, 0x3f, 0x59, + 0xb0, 0x18, 0x63, 0x2e, 0xa9, 0x1a, 0xf5, 0x70, 0x10, 0x68, 0xc9, 0xb0, 0x74, 0x48, 0x7f, 0xab, + 0x52, 0xe1, 0x5f, 0x47, 0x2b, 0xbd, 0x4b, 0xa5, 0x82, 0xef, 0x87, 0xaa, 0xdd, 0x12, 0x3c, 0xa2, + 0xd1, 0x60, 0xfa, 0xc3, 0x0f, 0x29, 0x89, 0x64, 0x6b, 0x3b, 0x5b, 0xe3, 0xbe, 0x59, 0xc2, 0xb5, + 0xe3, 0x53, 0x23, 0x8d, 0x7f, 0xce, 0x9a, 0xe6, 0x98, 0xc4, 0x13, 0xef, 0xee, 0x64, 0xfd, 0xc8, + 0xd2, 0x79, 0xff, 0xf1, 0x19, 0x9a, 0xf1, 0x66, 0xf7, 0xf9, 0x12, 0x6c, 0x1a, 0x49, 0xce, 0x82, + 0xc4, 0x27, 0x81, 0x67, 0xf0, 0x85, 0xcb, 0xe0, 0x17, 0xa6, 0xb0, 0x9e, 0x66, 0xba, 0x03, 0xf5, + 0x80, 0xec, 0xe1, 0x24, 0x94, 0x9e, 0x12, 0xc1, 0xa2, 0x2e, 0x10, 0x94, 0x4a, 0x3e, 0x6c, 0x18, + 0xd3, 0x8e, 0xdb, 0x75, 0x21, 0x9d, 0xb6, 0xc3, 0x29, 0xfa, 0xa3, 0x05, 0x1f, 0x24, 0x9c, 0x0a, + 0xaf, 0x3f, 0xf6, 0x42, 0xe6, 0xe3, 0x90, 0xca, 0xb1, 0x37, 0x3c, 0x74, 0x4a, 0xda, 0x85, 0x2f, + 0x2e, 0x6e, 0xf0, 0xd3, 0xbd, 0x2b, 0xf9, 0x14, 0xeb, 0xe3, 0xaf, 0x52, 0x86, 0x47, 0x87, 0x46, + 0xbd, 0xae, 0x1f, 0x1f, 0xad, 0xd8, 0x3b, 0x6e, 0x37, 0x6f, 0xda, 0x75, 0xed, 0xe4, 0xd4, 0x64, + 0xe4, 0x42, 0x7d, 0x74, 0xe8, 0xfb, 0xde, 0x1e, 0x0d, 0x25, 0xe1, 0xba, 0xa1, 0xce, 0x9f, 0xe8, + 0x96, 0xd9, 0xfe, 0xbf, 0xde, 0xed, 0x74, 0x36, 0xf5, 0xa4, 0xe9, 0xce, 0xa6, 0x63, 0x2e, 0x28, + 0x16, 0xf3, 0x8d, 0xbe, 0x04, 0x20, 0x91, 0xcf, 0xc7, 0xb1, 0x6e, 0x6a, 0xa6, 0xad, 0x34, 0xcf, + 0xa0, 0x54, 0x22, 0xfe, 0x60, 0x32, 0xf1, 0x89, 0xfe, 0x2b, 0xdc, 0x1c, 0x16, 0x3d, 0x81, 0xc5, + 0xbe, 0xde, 0xad, 0x27, 0x24, 0xe6, 0xd2, 0x53, 0x12, 0x79, 0x95, 0x8e, 0xbe, 0x60, 0xd0, 0x3d, + 0x05, 0x56, 0x36, 0xf4, 0x08, 0xd2, 0x21, 0x8f, 0x44, 0x81, 0xa1, 0xab, 0x5e, 0x9e, 0x6e, 0xce, + 0x60, 0x1f, 0x44, 0x81, 0x26, 0xdb, 0x81, 0x4a, 0x3c, 0xf4, 0x68, 0x90, 0xf5, 0x9a, 0x3b, 0x97, + 0x3e, 0xb3, 0xed, 0x61, 0x37, 0x48, 0xdb, 0x4c, 0x4d, 0xdd, 0x0a, 0xb6, 0x1f, 0x75, 0x37, 0x84, + 0x5b, 0x8e, 0xd5, 0xf0, 0x29, 0xb5, 0x85, 0xf7, 0xa5, 0xb6, 0x1d, 0xf8, 0xf0, 0xcc, 0xd4, 0x39, + 0x43, 0x00, 0xcf, 0x97, 0xa3, 0xbb, 0x00, 0xd3, 0xbd, 0xe4, 0x91, 0xa5, 0x33, 0x90, 0xd5, 0x1c, + 0xb2, 0xf1, 0x0f, 0x0b, 0x6c, 0x97, 0x08, 0xc9, 0x38, 0x51, 0x45, 0x64, 0x08, 0x3e, 0x83, 0x92, + 0xaa, 0xc3, 0x54, 0xf2, 0xdf, 0x52, 0x86, 0x7a, 0x2a, 0xba, 0x0f, 0xe5, 0x3d, 0xaa, 0xfa, 0xbd, + 0x29, 0xdd, 0xb5, 0xb3, 0xae, 0x0b, 0x5a, 0x82, 0x5d, 0x72, 0x90, 0x10, 0x21, 0x75, 0xd6, 0x65, + 0x42, 0xa0, 0x91, 0xe8, 0x06, 0xd4, 0xb3, 0x7b, 0x4c, 0x37, 0x78, 0xa1, 0xcb, 0x37, 0x6b, 0xcc, + 0x79, 0x43, 0xe3, 0x0f, 0x45, 0x58, 0x48, 0x5d, 0x9e, 0x28, 0xcf, 0x26, 0x5c, 0xe3, 0x66, 0xc8, + 0x64, 0x93, 0x75, 0xf9, 0x6c, 0xaa, 0xa7, 0x40, 0x9d, 0x4b, 0x27, 0x6b, 0xa6, 0xf0, 0x0e, 0x35, + 0xd3, 0x85, 0x0a, 0x27, 0xfa, 0x5a, 0x62, 0x2e, 0xe7, 0x3f, 0x79, 0x6b, 0x44, 0xd2, 0x3b, 0xfa, + 0x90, 0x8c, 0xb3, 0xcb, 0x94, 0x21, 0x50, 0x97, 0xa9, 0x34, 0xc1, 0x8d, 0x28, 0xfd, 0xf4, 0xa2, + 0xce, 0x77, 0x22, 0x2e, 0x17, 0x66, 0xf8, 0x3b, 0x64, 0xcd, 0x5f, 0x0b, 0xf0, 0x41, 0x2f, 0x0e, + 0xa9, 0xbc, 0x1f, 0x05, 0x3d, 0x1f, 0x4b, 0x99, 0xbe, 0x8e, 0x7e, 0x0f, 0x15, 0x7d, 0xfd, 0xcf, + 0x3a, 0xc0, 0xbd, 0xf3, 0x3d, 0x3d, 0x03, 0x9e, 0x79, 0xaf, 0xfd, 0xe9, 0x28, 0x9e, 0x2c, 0x10, + 0x86, 0x34, 0x17, 0xd3, 0xc2, 0x3b, 0xc6, 0x74, 0xc9, 0x83, 0xc5, 0x37, 0x56, 0x43, 0x5b, 0x30, + 0x4b, 0xd4, 0x6d, 0x9f, 0x64, 0xfe, 0xdf, 0x7c, 0x6b, 0xa4, 0x27, 0x45, 0x93, 0xf2, 0x67, 0x04, + 0x8d, 0x3f, 0x17, 0x61, 0xae, 0xd3, 0xdb, 0xfd, 0x86, 0xd3, 0x2c, 0x38, 0x37, 0x54, 0x7b, 0x12, + 0x92, 0x46, 0xe6, 0xa5, 0xa5, 0x0b, 0x3b, 0xcb, 0xc1, 0x9c, 0x01, 0xfd, 0x08, 0xae, 0x29, 0xa5, + 0xf0, 0x62, 0x1d, 0x18, 0x93, 0x85, 0x93, 0x89, 0x5a, 0x43, 0x8c, 0x01, 0x7d, 0x0e, 0xb3, 0xcc, + 0x64, 0x9e, 0x2e, 0x96, 0xfa, 0x99, 0x0d, 0xa3, 0xd3, 0xdb, 0x4d, 0xd3, 0x33, 0xf3, 0x30, 0xc5, + 0x4c, 0xdf, 0x70, 0x9c, 0x3d, 0x17, 0xe9, 0xc3, 0x2f, 0xff, 0x86, 0x73, 0xd9, 0x73, 0x81, 0x7e, + 0x07, 0x8b, 0x3e, 0x1b, 0xc5, 0xaa, 0xf6, 0xd4, 0x8d, 0xc3, 0x67, 0x01, 0xf1, 0xd3, 0xf6, 0x74, + 0xc1, 0x73, 0x53, 0x95, 0x47, 0x67, 0x0a, 0x4b, 0x69, 0xed, 0x1c, 0x53, 0x47, 0x11, 0x9d, 0xd2, + 0xd8, 0xca, 0x7b, 0xd2, 0xd8, 0xc6, 0x37, 0xb0, 0xb8, 0x9e, 0x84, 0x6a, 0x43, 0xb9, 0xe3, 0x98, + 0x3c, 0xc0, 0xad, 0xff, 0xfb, 0x01, 0x7e, 0x73, 0x0d, 0x16, 0x4e, 0x6d, 0x15, 0x55, 0xa1, 0xf4, + 0x98, 0x45, 0xc4, 0x9e, 0x51, 0x5f, 0x0f, 0x5f, 0xd2, 0xd8, 0xb6, 0xd6, 0x6f, 0xbd, 0xfa, 0xcf, + 0xf2, 0xcc, 0xab, 0xe3, 0x65, 0xeb, 0xbb, 0xe3, 0x65, 0xeb, 0xfb, 0xe3, 0x65, 0xeb, 0xdf, 0xc7, + 0xcb, 0xd6, 0x5f, 0x5e, 0x2f, 0xcf, 0x7c, 0xf7, 0x7a, 0x79, 0xe6, 0xfb, 0xd7, 0xcb, 0x33, 0xdf, + 0xd6, 0x73, 0xff, 0xc7, 0xf1, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x51, 0xaf, 0x2a, 0x1e, 0x90, + 0x11, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 0ae146ea4fd2..0f19bf7019e0 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -134,6 +134,10 @@ message ReadImportDataSpec { // NEXTID: 16 } +message StreamIngestionDataSpec { + repeated string partition_address = 1 [(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.PartitionAddress",(gogoproto.nullable) = false]; +} + message BackupDataSpec { repeated roachpb.Span spans = 1 [(gogoproto.nullable) = false]; repeated roachpb.Span introduced_spans = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 8dca196c74ea..6234f76741bf 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2291,6 +2291,7 @@ var charts = []sectionDescription{ "jobs.schema_change.currently_running", "jobs.schema_change_gc.currently_running", "jobs.typedesc_schema_change.currently_running", + "jobs.stream_ingestion.currently_running", }, }, { @@ -2401,6 +2402,17 @@ var charts = []sectionDescription{ }, Rate: DescribeDerivative_NON_NEGATIVE_DERIVATIVE, }, + { + Title: "Stream Ingestion", + Metrics: []string{ + "jobs.stream_ingestion.fail_or_cancel_completed", + "jobs.stream_ingestion.fail_or_cancel_failed", + "jobs.stream_ingestion.fail_or_cancel_retry_error", + "jobs.stream_ingestion.resume_completed", + "jobs.stream_ingestion.resume_failed", + "jobs.stream_ingestion.resume_retry_error", + }, + }, }, }, }