From 5089ab3bf6349686305de645f998b6154dda6dd8 Mon Sep 17 00:00:00 2001 From: David Taylor Date: Fri, 13 Aug 2021 14:24:40 +0000 Subject: [PATCH 1/5] backupccl: move SHOW BACKUP manifest loading to helper This pulls the manifest loading and rendering step of SHOW BACKUP into a helper, behind an interface, that can then allow for other implementations of the interface that load backup information differently, much like how the actual renderer was already plugable. Release note: none. --- pkg/ccl/backupccl/show.go | 119 ++++++++++++++++++++++++-------------- 1 file changed, 77 insertions(+), 42 deletions(-) diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index ef88d1d8788c..43f6de798f75 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -61,6 +61,79 @@ func checkShowBackupURIPrivileges(ctx context.Context, p sql.PlanHookState, uri return nil } +type backupInfoReader interface { + showBackup( + context.Context, + cloud.ExternalStorage, + *jobspb.BackupEncryptionOptions, + []string, + chan<- tree.Datums, + ) error + header() colinfo.ResultColumns +} + +type manifestInfoReader struct { + shower backupShower +} + +var _ backupInfoReader = manifestInfoReader{} + +func (m manifestInfoReader) header() colinfo.ResultColumns { + return m.shower.header +} + +func (m manifestInfoReader) showBackup( + ctx context.Context, + store cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + incPaths []string, + resultsCh chan<- tree.Datums, +) error { + var err error + manifests := make([]BackupManifest, len(incPaths)+1) + manifests[0], err = ReadBackupManifestFromStore(ctx, store, enc) + if err != nil { + return err + } + + for i := range incPaths { + m, err := readBackupManifest(ctx, store, incPaths[i], enc) + if err != nil { + return err + } + // Blank the stats to prevent memory blowup. + m.DeprecatedStatistics = nil + manifests[i+1] = m + } + + // Ensure that the descriptors in the backup manifests are up to date. + // + // This is necessary in particular for upgrading descriptors with old-style + // foreign keys which are no longer supported. + // If we are restoring a backup with old-style foreign keys, skip over the + // FKs for which we can't resolve the cross-table references. We can't + // display them anyway, because we don't have the referenced table names, + // etc. + err = maybeUpgradeDescriptorsInBackupManifests(ctx, manifests, true /* skipFKsWithNoMatchingTable */) + if err != nil { + return err + } + + datums, err := m.shower.fn(manifests) + if err != nil { + return err + } + + for _, row := range datums { + select { + case <-ctx.Done(): + return ctx.Err() + case resultsCh <- row: + } + } + return nil +} + // showBackupPlanHook implements PlanHookFn. func showBackupPlanHook( ctx context.Context, stmt tree.Statement, p sql.PlanHookState, @@ -107,6 +180,7 @@ func showBackupPlanHook( backup.Details = tree.BackupManifestAsJSON } + var infoReader backupInfoReader var shower backupShower switch backup.Details { case tree.BackupRangeDetails: @@ -118,6 +192,7 @@ func showBackupPlanHook( default: shower = backupShowerDefault(ctx, p, backup.ShouldIncludeSchemas, opts) } + infoReader = manifestInfoReader{shower} fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) error { // TODO(dan): Move this span into sql. @@ -190,50 +265,10 @@ func showBackupPlanHook( } } - manifests := make([]BackupManifest, len(incPaths)+1) - manifests[0], err = ReadBackupManifestFromStore(ctx, store, encryption) - if err != nil { - return err - } - - for i := range incPaths { - m, err := readBackupManifest(ctx, store, incPaths[i], encryption) - if err != nil { - return err - } - // Blank the stats to prevent memory blowup. - m.DeprecatedStatistics = nil - manifests[i+1] = m - } - - // Ensure that the descriptors in the backup manifests are up to date. - // - // This is necessary in particular for upgrading descriptors with old-style - // foreign keys which are no longer supported. - // If we are restoring a backup with old-style foreign keys, skip over the - // FKs for which we can't resolve the cross-table references. We can't - // display them anyway, because we don't have the referenced table names, - // etc. - err = maybeUpgradeDescriptorsInBackupManifests(ctx, manifests, true /* skipFKsWithNoMatchingTable */) - if err != nil { - return err - } - - datums, err := shower.fn(manifests) - if err != nil { - return err - } - for _, row := range datums { - select { - case <-ctx.Done(): - return ctx.Err() - case resultsCh <- row: - } - } - return nil + return infoReader.showBackup(ctx, store, encryption, incPaths, resultsCh) } - return fn, shower.header, nil, false, nil + return fn, infoReader.header(), nil, false, nil } type backupShower struct { From edc8951b264276ff976d8a2f8b8486396b262bc8 Mon Sep 17 00:00:00 2001 From: Eugene Kalinin Date: Mon, 9 Aug 2021 15:42:43 +0300 Subject: [PATCH 2/5] sql: support COMMENT ON SCHEMA This change adds support for SCHEMA commenting. Release note (sql change): This change adds associating comment to SQL schema using PostgreSQL's COMMENT ON SCHEMA syntax. --- docs/generated/sql/bnf/comment.bnf | 1 + docs/generated/sql/bnf/stmt_block.bnf | 7 +- pkg/ccl/importccl/read_import_pgdump.go | 6 +- pkg/keys/constants.go | 1 + pkg/sql/BUILD.bazel | 1 + pkg/sql/comment_on_schema.go | 95 +++++++++++++++++++ pkg/sql/drop_schema.go | 18 ++++ .../logictest/testdata/logic_test/pg_catalog | 11 +++ pkg/sql/logictest/testdata/logic_test/schema | 32 +++++++ pkg/sql/opaque.go | 3 + pkg/sql/parser/sql.y | 4 + pkg/sql/pg_catalog.go | 3 + pkg/sql/plan_opt.go | 2 +- pkg/sql/sem/builtins/pg_builtins.go | 2 + pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/comment_on_schema.go | 37 ++++++++ pkg/sql/sem/tree/stmt.go | 10 ++ pkg/sql/walk.go | 1 + 18 files changed, 229 insertions(+), 6 deletions(-) create mode 100644 pkg/sql/comment_on_schema.go create mode 100644 pkg/sql/sem/tree/comment_on_schema.go diff --git a/docs/generated/sql/bnf/comment.bnf b/docs/generated/sql/bnf/comment.bnf index 31ac4179e833..a18d57fe3ce4 100644 --- a/docs/generated/sql/bnf/comment.bnf +++ b/docs/generated/sql/bnf/comment.bnf @@ -1,5 +1,6 @@ comment_stmt ::= 'COMMENT' 'ON' 'DATABASE' database_name 'IS' comment_text + | 'COMMENT' 'ON' 'SCHEMA' schema_name 'IS' comment_text | 'COMMENT' 'ON' 'TABLE' table_name 'IS' comment_text | 'COMMENT' 'ON' 'COLUMN' column_name 'IS' comment_text | 'COMMENT' 'ON' 'INDEX' table_index_name 'IS' comment_text diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index b6386511c91d..27c250564edf 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -55,6 +55,7 @@ copy_from_stmt ::= comment_stmt ::= 'COMMENT' 'ON' 'DATABASE' database_name 'IS' comment_text + | 'COMMENT' 'ON' 'SCHEMA' schema_name 'IS' comment_text | 'COMMENT' 'ON' 'TABLE' table_name 'IS' comment_text | 'COMMENT' 'ON' 'COLUMN' column_path 'IS' comment_text | 'COMMENT' 'ON' 'INDEX' table_index_name 'IS' comment_text @@ -272,6 +273,9 @@ comment_text ::= 'SCONST' | 'NULL' +schema_name ::= + name + column_path ::= name | prefixed_column_path @@ -1806,9 +1810,6 @@ alter_zone_partition_stmt ::= | 'ALTER' 'PARTITION' partition_name 'OF' 'INDEX' table_index_name set_zone_config | 'ALTER' 'PARTITION' partition_name 'OF' 'INDEX' table_name '@' '*' set_zone_config -schema_name ::= - name - opt_add_val_placement ::= 'BEFORE' 'SCONST' | 'AFTER' 'SCONST' diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 2c6c86c987a4..6fd2ad502cae 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -850,7 +850,8 @@ func readPostgresStmt( case *tree.Insert, *tree.CopyFrom, *tree.Delete, copyData: // handled during the data ingestion pass. case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, - *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.SetVar, *tree.Analyze: + *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.SetVar, *tree.Analyze, + *tree.CommentOnSchema: // These are the statements that can be parsed by CRDB but are not // supported, or are not required to be processed, during an IMPORT. // - ignore txns. @@ -1344,7 +1345,8 @@ func (m *pgDumpReader) readFile( return wrapErrorWithUnsupportedHint(err) } case *tree.CreateExtension, *tree.CommentOnDatabase, *tree.CommentOnTable, - *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence: + *tree.CommentOnIndex, *tree.CommentOnColumn, *tree.AlterSequence, + *tree.CommentOnSchema: // handled during schema extraction. case *tree.SetVar, *tree.BeginTransaction, *tree.CommitTransaction, *tree.Analyze: // handled during schema extraction. diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 2c4ce11a0006..bf61c547d8d1 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -416,6 +416,7 @@ const ( TableCommentType = 1 ColumnCommentType = 2 IndexCommentType = 3 + SchemaCommentType = 4 ) const ( diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index c5eb99b5a6d8..c723e0726e3d 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "comment_on_column.go", "comment_on_database.go", "comment_on_index.go", + "comment_on_schema.go", "comment_on_table.go", "conn_executor.go", "conn_executor_exec.go", diff --git a/pkg/sql/comment_on_schema.go b/pkg/sql/comment_on_schema.go new file mode 100644 index 000000000000..9e2095dd5383 --- /dev/null +++ b/pkg/sql/comment_on_schema.go @@ -0,0 +1,95 @@ +// Copyright 2021 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 sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" +) + +type commentOnSchemaNode struct { + n *tree.CommentOnSchema + schemaDesc catalog.SchemaDescriptor +} + +// CommentOnSchema add comment on a schema. +// Privileges: CREATE on scheme. +// notes: postgres requires CREATE on the scheme. +func (p *planner) CommentOnSchema(ctx context.Context, n *tree.CommentOnSchema) (planNode, error) { + if err := checkSchemaChangeEnabled( + ctx, + p.ExecCfg(), + "COMMENT ON SCHEMA", + ); err != nil { + return nil, err + } + + // Users can't create a schema without being connected to a DB. + dbName := p.CurrentDatabase() + if dbName == "" { + return nil, pgerror.New(pgcode.UndefinedDatabase, + "cannot comment schema without being connected to a database") + } + + db, err := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, + dbName, tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return nil, err + } + + schemaDesc, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, + db.GetSchemaID(string(n.Name)), tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return nil, err + } + + if err := p.CheckPrivilege(ctx, db, privilege.CREATE); err != nil { + return nil, err + } + + return &commentOnSchemaNode{n: n, schemaDesc: schemaDesc}, nil +} + +func (n *commentOnSchemaNode) startExec(params runParams) error { + if n.n.Comment != nil { + _, err := params.p.extendedEvalCtx.ExecCfg.InternalExecutor.ExecEx( + params.ctx, + "set-schema-comment", + params.p.Txn(), + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "UPSERT INTO system.comments VALUES ($1, $2, 0, $3)", + keys.SchemaCommentType, + n.schemaDesc.GetID(), + *n.n.Comment) + if err != nil { + return err + } + } else { + err := params.p.removeSchemaComment(params.ctx, n.schemaDesc.GetID()) + if err != nil { + return err + } + } + + return nil +} + +func (n *commentOnSchemaNode) Next(runParams) (bool, error) { return false, nil } +func (n *commentOnSchemaNode) Values() tree.Datums { return tree.Datums{} } +func (n *commentOnSchemaNode) Close(context.Context) {} diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index 1aa20005fb1c..515824834167 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" @@ -212,6 +214,9 @@ func (p *planner) dropSchemaImpl( } // Mark the descriptor as dropped. sc.State = descpb.DescriptorState_DROP + if err := p.removeSchemaComment(ctx, sc.GetID()); err != nil { + return err + } return p.writeSchemaDesc(ctx, sc) } @@ -245,6 +250,19 @@ func (p *planner) createDropSchemaJob( return err } +func (p *planner) removeSchemaComment(ctx context.Context, schemaID descpb.ID) error { + _, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.ExecEx( + ctx, + "delete-schema-comment", + p.txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=0", + keys.SchemaCommentType, + schemaID) + + return err +} + func (n *dropSchemaNode) Next(params runParams) (bool, error) { return false, nil } func (n *dropSchemaNode) Values() tree.Datums { return tree.Datums{} } func (n *dropSchemaNode) Close(ctx context.Context) {} diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 4f106f71ac0c..cb4af9ba0fd1 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -5074,3 +5074,14 @@ SELECT sequenceowner, cache_size FROM pg_sequences WHERE sequencename = 'serial' ---- sequenceowner cache_size anyuser 10 + +statement ok +COMMENT ON SCHEMA test_schema is 'testing schema'; + +query T colnames +SELECT obj_description(objoid) + FROM pg_catalog.pg_description + WHERE description = 'testing schema' +---- +obj_description +testing schema diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index fe2f002f568f..3b39bff032ef 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -868,3 +868,35 @@ SHOW TABLES statement ok DROP DATABASE samename CASCADE; + +# Verify schema comments +subtest schema_comments + +user root + +statement ok +CREATE DATABASE comment_db + +statement ok +USE comment_db + +statement ok +CREATE SCHEMA foo + +statement ok +COMMENT ON SCHEMA foo IS 'bar' + +query T +SELECT comment FROM system.comments LIMIT 1 +---- +bar + +statement ok +DROP SCHEMA foo + +query T +SELECT comment FROM system.comments LIMIT 1 +---- + +statement ok +DROP DATABASE comment_db diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 76e7a83b2ce5..7ad640f8f5a1 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -112,6 +112,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.CommentOnColumn(ctx, n) case *tree.CommentOnDatabase: return p.CommentOnDatabase(ctx, n) + case *tree.CommentOnSchema: + return p.CommentOnSchema(ctx, n) case *tree.CommentOnIndex: return p.CommentOnIndex(ctx, n) case *tree.CommentOnTable: @@ -238,6 +240,7 @@ func init() { &tree.AlterRoleSet{}, &tree.CommentOnColumn{}, &tree.CommentOnDatabase{}, + &tree.CommentOnSchema{}, &tree.CommentOnIndex{}, &tree.CommentOnTable{}, &tree.CreateDatabase{}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 01e4e62f2d57..7fef95d8200e 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -3342,6 +3342,10 @@ comment_stmt: { $$.val = &tree.CommentOnDatabase{Name: tree.Name($4), Comment: $6.strPtr()} } +| COMMENT ON SCHEMA schema_name IS comment_text + { + $$.val = &tree.CommentOnSchema{Name: tree.Name($4), Comment: $6.strPtr()} + } | COMMENT ON TABLE table_name IS comment_text { $$.val = &tree.CommentOnTable{Table: $4.unresolvedObjectName(), Comment: $6.strPtr()} diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index f96d9dcde478..f027393f7fcb 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -1390,6 +1390,9 @@ https://www.postgresql.org/docs/9.5/catalog-pg-description.html`, case keys.DatabaseCommentType: // Database comments are exported in pg_shdescription. continue + case keys.SchemaCommentType: + objID = tree.NewDOid(tree.MustBeDInt(objID)) + classOid = tree.NewDOid(catconstants.PgCatalogNamespaceTableID) case keys.ColumnCommentType, keys.TableCommentType: objID = tree.NewDOid(tree.MustBeDInt(objID)) classOid = tree.NewDOid(catconstants.PgCatalogClassTableID) diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 4309dec00fb1..13ffb0b535ee 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -55,7 +55,7 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error) case *tree.AlterIndex, *tree.AlterTable, *tree.AlterSequence, *tree.Analyze, *tree.BeginTransaction, - *tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable, + *tree.CommentOnColumn, *tree.CommentOnDatabase, *tree.CommentOnIndex, *tree.CommentOnTable, *tree.CommentOnSchema, *tree.CommitTransaction, *tree.CopyFrom, *tree.CreateDatabase, *tree.CreateIndex, *tree.CreateView, *tree.CreateSequence, diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index deb1d256c7f8..6a27c29aa1b4 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -1942,6 +1942,8 @@ func getCatalogOidForComments(catalogName string) (id int, ok bool) { return catconstants.PgCatalogClassTableID, true case "pg_database": return catconstants.PgCatalogDatabaseTableID, true + case "pg_description": + return catconstants.PgCatalogDescriptionTableID, true default: // We currently only support comments on pg_class objects // (columns, tables) in this context. diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index 5f5b5525a441..5d68d94f4f71 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "comment_on_column.go", "comment_on_database.go", "comment_on_index.go", + "comment_on_schema.go", "comment_on_table.go", "constant.go", "constant_eval.go", diff --git a/pkg/sql/sem/tree/comment_on_schema.go b/pkg/sql/sem/tree/comment_on_schema.go new file mode 100644 index 000000000000..eae57975b20d --- /dev/null +++ b/pkg/sql/sem/tree/comment_on_schema.go @@ -0,0 +1,37 @@ +// Copyright 2021 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 tree + +import "github.com/cockroachdb/cockroach/pkg/sql/lexbase" + +// CommentOnSchema represents an COMMENT ON SCHEMA statement. +type CommentOnSchema struct { + Name Name + Comment *string +} + +// Format implements the NodeFormatter interface. +func (n *CommentOnSchema) Format(ctx *FmtCtx) { + ctx.WriteString("COMMENT ON SCHEMA ") + ctx.FormatNode(&n.Name) + ctx.WriteString(" IS ") + if n.Comment != nil { + // TODO(knz): Replace all this with ctx.FormatNode + // when COMMENT supports expressions. + if ctx.flags.HasFlags(FmtHideConstants) { + ctx.WriteByte('_') + } else { + lexbase.EncodeSQLStringWithFlags(&ctx.Buffer, *n.Comment, ctx.flags.EncodeFlags()) + } + } else { + ctx.WriteString("NULL") + } +} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 9e5cb2a1f0b3..54dc5e716d4f 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -479,6 +479,15 @@ func (*CommentOnDatabase) StatementType() StatementType { return TypeDDL } // StatementTag returns a short string identifying the type of statement. func (*CommentOnDatabase) StatementTag() string { return "COMMENT ON DATABASE" } +// StatementReturnType implements the Statement interface. +func (*CommentOnSchema) StatementReturnType() StatementReturnType { return DDL } + +// StatementType implements the Statement interface. +func (*CommentOnSchema) StatementType() StatementType { return TypeDDL } + +// StatementTag returns a short string identifying the type of statement. +func (*CommentOnSchema) StatementTag() string { return "COMMENT ON SCHEMA" } + // StatementReturnType implements the Statement interface. func (*CommentOnIndex) StatementReturnType() StatementReturnType { return DDL } @@ -1622,6 +1631,7 @@ func (n *CancelSessions) String() string { return AsString(n) } func (n *CannedOptPlan) String() string { return AsString(n) } func (n *CommentOnColumn) String() string { return AsString(n) } func (n *CommentOnDatabase) String() string { return AsString(n) } +func (n *CommentOnSchema) String() string { return AsString(n) } func (n *CommentOnIndex) String() string { return AsString(n) } func (n *CommentOnTable) String() string { return AsString(n) } func (n *CommitTransaction) String() string { return AsString(n) } diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 44b785a49674..8cff6d0b9feb 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -358,6 +358,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&commentOnDatabaseNode{}): "comment on database", reflect.TypeOf(&commentOnIndexNode{}): "comment on index", reflect.TypeOf(&commentOnTableNode{}): "comment on table", + reflect.TypeOf(&commentOnSchemaNode{}): "comment on schema", reflect.TypeOf(&controlJobsNode{}): "control jobs", reflect.TypeOf(&controlSchedulesNode{}): "control schedules", reflect.TypeOf(&createDatabaseNode{}): "create database", From a017cc7e6abc97a97fddefef987c195e2dddf097 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 16 Jul 2021 14:16:22 -0400 Subject: [PATCH 3/5] sql: rename utility function in fetcher Release note: None --- pkg/sql/row/kv_batch_fetcher.go | 37 +++++++++++++++++---------------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index cfdd208d50ba..1a65b02a0f06 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -74,11 +74,11 @@ type txnKVFetcher struct { // single-key fetch and may use a GetRequest under the hood. spans roachpb.Spans // If useBatchLimit is true, batches are limited to kvBatchSize. If - // firstBatchLimit is also set, the first batch is limited to that value. + // firstBatchKeyLimit is also set, the first batch is limited to that value. // Subsequent batches are larger, up to kvBatchSize. - firstBatchLimit int64 - useBatchLimit bool - reverse bool + firstBatchKeyLimit int64 + useBatchLimit bool + reverse bool // lockStrength represents the locking mode to use when fetching KVs. lockStrength descpb.ScanLockingStrength // lockWaitPolicy represents the policy to be used for handling conflicting @@ -113,18 +113,19 @@ type txnKVFetcher struct { var _ kvBatchFetcher = &txnKVFetcher{} -// getBatchSize returns the max size of the next batch. The size is expressed in -// number of result keys (i.e. this size will be used for MaxSpanRequestKeys). -func (f *txnKVFetcher) getBatchSize() int64 { - return f.getBatchSizeForIdx(f.batchIdx) +// getBatchKeyLimit returns the max size of the next batch. The size is +// expressed in number of result keys (i.e. this size will be used for +// MaxSpanRequestKeys). +func (f *txnKVFetcher) getBatchKeyLimit() int64 { + return f.getBatchKeyLimitForIdx(f.batchIdx) } -func (f *txnKVFetcher) getBatchSizeForIdx(batchIdx int) int64 { +func (f *txnKVFetcher) getBatchKeyLimitForIdx(batchIdx int) int64 { if !f.useBatchLimit { return 0 } kvBatchSize := getKVBatchSize(f.forceProductionKVBatchSize) - if f.firstBatchLimit == 0 || f.firstBatchLimit >= kvBatchSize { + if f.firstBatchKeyLimit == 0 || f.firstBatchKeyLimit >= kvBatchSize { return kvBatchSize } @@ -133,7 +134,7 @@ func (f *txnKVFetcher) getBatchSizeForIdx(batchIdx int) int64 { // we revert to the default batch size. switch batchIdx { case 0: - return f.firstBatchLimit + return f.firstBatchKeyLimit case 1: // Make the second batch 10 times larger (but at most the default batch @@ -146,7 +147,7 @@ func (f *txnKVFetcher) getBatchSizeForIdx(batchIdx int) int64 { // 100 | 1,000 | 10,000 // 500 | 5,000 | 10,000 // 1000 | 10,000 | 10,000 - secondBatch := f.firstBatchLimit * 10 + secondBatch := f.firstBatchKeyLimit * 10 switch { case secondBatch < kvBatchSize/10: return kvBatchSize / 10 @@ -223,7 +224,7 @@ func makeKVBatchFetcherDefaultSendFunc(txn *kv.Txn) sendFunc { // makeKVBatchFetcher initializes a kvBatchFetcher for the given spans. If // useBatchLimit is true, the number of result keys per batch is limited; the -// limit grows between subsequent batches, starting at firstBatchLimit (if not +// limit grows between subsequent batches, starting at firstBatchKeyLimit (if not // 0) to productionKVBatchSize. // // Batch limits can only be used if the spans are ordered. @@ -232,7 +233,7 @@ func makeKVBatchFetcher( spans roachpb.Spans, reverse bool, useBatchLimit bool, - firstBatchLimit int64, + firstBatchKeyLimit int64, lockStrength descpb.ScanLockingStrength, lockWaitPolicy descpb.ScanLockingWaitPolicy, lockTimeout time.Duration, @@ -241,9 +242,9 @@ func makeKVBatchFetcher( requestAdmissionHeader roachpb.AdmissionHeader, responseAdmissionQ *admission.WorkQueue, ) (txnKVFetcher, error) { - if firstBatchLimit < 0 || (!useBatchLimit && firstBatchLimit != 0) { + if firstBatchKeyLimit < 0 || (!useBatchLimit && firstBatchKeyLimit != 0) { return txnKVFetcher{}, errors.Errorf("invalid batch limit %d (useBatchLimit: %t)", - firstBatchLimit, useBatchLimit) + firstBatchKeyLimit, useBatchLimit) } if useBatchLimit { @@ -302,7 +303,7 @@ func makeKVBatchFetcher( spans: copySpans, reverse: reverse, useBatchLimit: useBatchLimit, - firstBatchLimit: firstBatchLimit, + firstBatchKeyLimit: firstBatchKeyLimit, lockStrength: lockStrength, lockWaitPolicy: lockWaitPolicy, lockTimeout: lockTimeout, @@ -323,7 +324,7 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { var ba roachpb.BatchRequest ba.Header.WaitPolicy = f.getWaitPolicy() ba.Header.LockTimeout = f.lockTimeout - ba.Header.MaxSpanRequestKeys = f.getBatchSize() + ba.Header.MaxSpanRequestKeys = f.getBatchKeyLimit() if ba.Header.MaxSpanRequestKeys > 0 { // If this kvfetcher limits the number of rows returned, also use // target bytes to guard against the case in which the average row From bc0d4b9297936c021029185d7e75d408ef774e9c Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 16 Jul 2021 14:34:50 -0400 Subject: [PATCH 4/5] sql: rename a fetcher function Release note: None --- pkg/sql/row/fetcher.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 8d8359961baf..c82b4b0220ca 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -584,7 +584,7 @@ func (rf *Fetcher) StartScan( spans, rf.reverse, limitBatches, - rf.firstBatchLimit(limitHint), + rf.rowLimitToKeyLimit(limitHint), rf.lockStrength, rf.lockWaitPolicy, rf.lockTimeout, @@ -684,7 +684,7 @@ func (rf *Fetcher) StartInconsistentScan( spans, rf.reverse, limitBatches, - rf.firstBatchLimit(limitHint), + rf.rowLimitToKeyLimit(limitHint), rf.lockStrength, rf.lockWaitPolicy, rf.lockTimeout, @@ -699,19 +699,19 @@ func (rf *Fetcher) StartInconsistentScan( return rf.StartScanFrom(ctx, &f) } -func (rf *Fetcher) firstBatchLimit(limitHint int64) int64 { - if limitHint == 0 { +func (rf *Fetcher) rowLimitToKeyLimit(keyLimitHint int64) int64 { + if keyLimitHint == 0 { return 0 } // If we have a limit hint, we limit the first batch size. Subsequent // batches get larger to avoid making things too slow (e.g. in case we have // a very restrictive filter and actually have to retrieve a lot of rows). - // The limitHint is a row limit, but each row could be made up of more than + // The keyLimitHint is a row limit, but each row could be made up of more than // one key. We take the maximum possible keys per row out of all the table // rows we could potentially scan over. // // We add an extra key to make sure we form the last row. - return limitHint*int64(rf.maxKeysPerRow) + 1 + return keyLimitHint*int64(rf.maxKeysPerRow) + 1 } // StartScanFrom initializes and starts a scan from the given kvBatchFetcher. Can be From 0fa923a65cb525d865c1baf2da5acfabe82c4f02 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 16 Jul 2021 14:50:53 -0400 Subject: [PATCH 5/5] sql: rationalize batch key and bytes limit Before this patch, the txnKVFetcher had two modes of operation: either use both key limits and bytes limits, or use neither. This patch changes it to the following three modes: only bytes limits, both, neither - thus adding the only bytes limits mode. The idea is that, frequently, only bytes limits are desired - getting row limits in those cases was a historical artifact. The choices for the user of the fetcher are the following: 1) I want DistSender-level parallelism. I can't use any limits. 2) I want to read some key spans fully. But I don't want to OOM either myself (the client) or the server. In this case I want only memory limits (and I was getting row limits before this patch for no reason). 3) I hope I won't have to read my spans fully. The high-level query has a LIMIT, which has been processed into a fetcher-level hint about how many keys need to be read to satisfy the LIMIT. Even if that hint proves too small, there's still likely a point where enough rows will have been read. In this case, I start with my hint, and then (while insufficient), I progressively ratched it up. All the batches have bytes limits too since I don't want OOMs and, since I use key limits, DistSender-parallelism is inhibited anyway (so there's no reason not to use bytes limit). For case 3), there's still an upper-bound key limit to the "ratcheting up" behavior. This is productionKVBatchSize - which this patch increases from 10k keys to 100k. I'm not sure if the existence of this upper bound is rational; I'm thinking that it acts as an upper-bound on the amount of wasted work due to over-shooting the real LIMIT. Since the bytes limit is 10MB, it doesn't matter very much. The fetcher clients that don't specify a "firstBatchRowHint" used to get the bytes limit and a key limit of 10k. Now they get only the bytes limit. This category includes the joinReader, the inverted joiner, some backfills, and some DELETEs. For the fetcher client that specify a hint, the behavior is unchanged with the exception of the fact that the upper bound of the key limit ratcheting is now 100k instead of 10k. This means that, for rows under 1KB, they're now more likely to hit the memory limit then the key limit. This category includes the cFetcher, the TableReader, and the zig-zag joiner. Release note: None --- pkg/sql/backfill.go | 2 +- pkg/sql/backfill/backfill.go | 8 +- pkg/sql/colfetcher/cfetcher.go | 12 +- pkg/sql/colfetcher/colbatch_scan.go | 46 +- pkg/sql/colfetcher/index_join.go | 4 +- pkg/sql/conn_executor_test.go | 29 +- pkg/sql/create_stats_test.go | 16 +- pkg/sql/distsql_physical_planner.go | 4 + pkg/sql/execinfra/server_config.go | 9 + pkg/sql/execinfrapb/processors_sql.pb.go | 435 ++++++++++-------- pkg/sql/execinfrapb/processors_sql.proto | 12 + pkg/sql/indexbackfiller_test.go | 2 +- pkg/sql/logictest/BUILD.bazel | 1 - pkg/sql/logictest/logic.go | 21 - .../logic_test/select_index_span_ranges | 66 --- .../testdata/lookup_join_nonmetamorphic | 68 +-- pkg/sql/row/fetcher.go | 61 ++- pkg/sql/row/fetcher_test.go | 20 +- pkg/sql/row/kv_batch_fetcher.go | 90 ++-- pkg/sql/row/kv_fetcher.go | 6 +- pkg/sql/rowexec/BUILD.bazel | 1 + pkg/sql/rowexec/backfiller.go | 5 +- pkg/sql/rowexec/columnbackfiller.go | 3 +- pkg/sql/rowexec/inverted_joiner.go | 2 +- pkg/sql/rowexec/joinreader.go | 26 +- pkg/sql/rowexec/joinreader_blackbox_test.go | 99 ++++ pkg/sql/rowexec/rowfetcher.go | 8 +- pkg/sql/rowexec/scrub_tablereader.go | 4 +- pkg/sql/rowexec/stats.go | 12 +- pkg/sql/rowexec/tablereader.go | 37 +- pkg/sql/rowexec/zigzagjoiner.go | 6 +- pkg/sql/scan_test.go | 21 +- pkg/sql/stats/BUILD.bazel | 2 +- pkg/sql/stats/create_stats_job_test.go | 22 +- pkg/sql/tablewriter_delete.go | 4 +- pkg/testutils/lint/lint_test.go | 1 + pkg/util/tracing/test_utils.go | 17 +- 37 files changed, 681 insertions(+), 501 deletions(-) delete mode 100644 pkg/sql/logictest/testdata/logic_test/select_index_span_ranges create mode 100644 pkg/sql/rowexec/joinreader_blackbox_test.go diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 76633df7d470..e063fa9d2676 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -2411,7 +2411,7 @@ func columnBackfillInTxn( for sp.Key != nil { var err error sp.Key, err = backfiller.RunColumnBackfillChunk(ctx, - txn, tableDesc, sp, columnBackfillBatchSize.Get(&evalCtx.Settings.SV), + txn, tableDesc, sp, row.RowLimit(columnBackfillBatchSize.Get(&evalCtx.Settings.SV)), false /*alsoCommit*/, traceKV) if err != nil { return err diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 4d690ce0df87..d2b1927a5482 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -247,7 +247,7 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( txn *kv.Txn, tableDesc catalog.TableDescriptor, sp roachpb.Span, - chunkSize int64, + chunkSize row.RowLimit, alsoCommit bool, traceKV bool, ) (roachpb.Key, error) { @@ -287,7 +287,7 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( // populated and deleted by the OLTP commands but not otherwise // read or used if err := cb.fetcher.StartScan( - ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, chunkSize, + ctx, txn, []roachpb.Span{sp}, row.DefaultBatchBytesLimit, chunkSize, traceKV, false, /* forceProductionKVBatchSize */ ); err != nil { log.Errorf(ctx, "scan error: %s", err) @@ -305,7 +305,7 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( iv.Cols = append(iv.Cols, tableDesc.PublicColumns()...) iv.Cols = append(iv.Cols, cb.added...) cb.evalCtx.IVarContainer = iv - for i := int64(0); i < chunkSize; i++ { + for i := int64(0); i < int64(chunkSize); i++ { datums, _, _, err := cb.fetcher.NextRowDecoded(ctx) if err != nil { return roachpb.Key{}, err @@ -809,7 +809,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk( } defer fetcher.Close(ctx) if err := fetcher.StartScan( - ctx, txn, []roachpb.Span{sp}, true /* limitBatches */, initBufferSize, + ctx, txn, []roachpb.Span{sp}, row.DefaultBatchBytesLimit, initBufferSize, traceKV, false, /* forceProductionKVBatchSize */ ); err != nil { log.Errorf(ctx, "scan error: %s", err) diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 8b3529f44a09..fb65751237bc 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -664,26 +664,30 @@ func (rf *cFetcher) StartScan( spans roachpb.Spans, bsHeader *roachpb.BoundedStalenessHeader, limitBatches bool, - limitHint int64, + batchBytesLimit row.BytesLimit, + limitHint row.RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error { if len(spans) == 0 { return errors.AssertionFailedf("no spans") } + if !limitBatches && batchBytesLimit != row.NoBytesLimit { + return errors.AssertionFailedf("batchBytesLimit set without limitBatches") + } rf.traceKV = traceKV // If we have a limit hint, we limit the first batch size. Subsequent // batches get larger to avoid making things too slow (e.g. in case we have // a very restrictive filter and actually have to retrieve a lot of rows). - firstBatchLimit := limitHint + firstBatchLimit := row.KeyLimit(limitHint) if firstBatchLimit != 0 { // The limitHint is a row limit, but each row could be made up // of more than one key. We take the maximum possible keys // per row out of all the table rows we could potentially // scan over. - firstBatchLimit = limitHint * int64(rf.maxKeysPerRow) + firstBatchLimit = row.KeyLimit(int(limitHint) * rf.maxKeysPerRow) // We need an extra key to make sure we form the last row. firstBatchLimit++ } @@ -695,7 +699,7 @@ func (rf *cFetcher) StartScan( spans, bsHeader, rf.reverse, - limitBatches, + batchBytesLimit, firstBatchLimit, rf.lockStrength, rf.lockWaitPolicy, diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 22ff60043fa4..de5a394d13db 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colmem" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" @@ -47,12 +48,13 @@ type ColBatchScan struct { colexecop.ZeroInputNode colexecop.InitHelper - spans roachpb.Spans - flowCtx *execinfra.FlowCtx - bsHeader *roachpb.BoundedStalenessHeader - rf *cFetcher - limitHint int64 - parallelize bool + spans roachpb.Spans + flowCtx *execinfra.FlowCtx + bsHeader *roachpb.BoundedStalenessHeader + rf *cFetcher + limitHint row.RowLimit + batchBytesLimit row.BytesLimit + parallelize bool // tracingSpan is created when the stats should be collected for the query // execution, and it will be finished when closing the operator. tracingSpan *tracing.Span @@ -94,6 +96,7 @@ func (s *ColBatchScan) Init(ctx context.Context) { s.spans, s.bsHeader, limitBatches, + s.batchBytesLimit, s.limitHint, s.flowCtx.TraceKV, s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes, @@ -191,7 +194,7 @@ func NewColBatchScan( return nil, errors.AssertionFailedf("attempting to create a cFetcher with the IsCheck flag set") } - limitHint := execinfra.LimitHint(spec.LimitHint, post) + limitHint := row.RowLimit(execinfra.LimitHint(spec.LimitHint, post)) // TODO(ajwerner): The need to construct an immutable here // indicates that we're probably doing this wrong. Instead we should be // just setting the ID and Version in the spec or something like that and @@ -249,16 +252,29 @@ func NewColBatchScan( //gcassert:bce spans = append(spans, specSpans[i].Span) } - *s = ColBatchScan{ - spans: spans, - flowCtx: flowCtx, - bsHeader: bsHeader, - rf: fetcher, - limitHint: limitHint, + + if spec.LimitHint > 0 || spec.BatchBytesLimit > 0 { // Parallelize shouldn't be set when there's a limit hint, but double-check // just in case. - parallelize: spec.Parallelize && limitHint == 0, - ResultTypes: typs, + spec.Parallelize = false + } + var batchBytesLimit row.BytesLimit + if !spec.Parallelize { + batchBytesLimit = row.BytesLimit(spec.BatchBytesLimit) + if batchBytesLimit == 0 { + batchBytesLimit = row.DefaultBatchBytesLimit + } + } + + *s = ColBatchScan{ + spans: spans, + flowCtx: flowCtx, + bsHeader: bsHeader, + rf: fetcher, + limitHint: limitHint, + batchBytesLimit: batchBytesLimit, + parallelize: spec.Parallelize, + ResultTypes: typs, } return s, nil } diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index 09a38ea5ff24..5e677400751a 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/memsize" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -171,7 +172,8 @@ func (s *ColIndexJoin) Next() coldata.Batch { spans, nil, /* bsHeader */ false, /* limitBatches */ - 0, /* limitHint */ + row.NoBytesLimit, + row.NoRowLimit, s.flowCtx.TraceKV, s.flowCtx.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ); err != nil { diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index e9f7742307af..6675f15080f6 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -32,10 +32,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/mutations" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -408,8 +408,7 @@ func TestHalloweenProblemAvoidance(t *testing.T) { // We are also careful to override these defaults before starting // the server, so as to not risk updating them concurrently with // some background SQL activity. - const smallerKvBatchSize = 10 - defer row.TestingSetKVBatchSize(smallerKvBatchSize)() + const smallerKvBatchSize = 5 // This is an approximation based on the TableReaderBatchBytesLimit below. const smallerInsertBatchSize = 5 mutations.SetMaxBatchSizeForTests(smallerInsertBatchSize) defer mutations.ResetMaxBatchSizeForTests() @@ -417,6 +416,10 @@ func TestHalloweenProblemAvoidance(t *testing.T) { params, _ := tests.CreateTestServerParams() params.Insecure = true + params.Knobs.DistSQL = &execinfra.TestingKnobs{ + TableReaderBatchBytesLimit: 10, + } + s, db, _ := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) @@ -508,13 +511,12 @@ func TestQueryProgress(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - const rows, kvBatchSize = 1000, 50 - + const rows = 1000 defer rowexec.TestingSetScannedRowProgressFrequency(rows / 60)() - defer row.TestingSetKVBatchSize(kvBatchSize)() - const expectedScans = (rows / 2) /* WHERE restricts scan to 1/2 */ / kvBatchSize - const stallAfterScans = expectedScans/2 + 1 + // We'll do more than 6 scans because we set a low TableReaderBatchBytesLimit + // below. + const stallAfterScans = 6 var queryRunningAtomic, scannedBatchesAtomic int64 stalled, unblock := make(chan struct{}), make(chan struct{}) @@ -531,6 +533,11 @@ func TestQueryProgress(t *testing.T) { // then close once it has checked the progress). params := base.TestServerArgs{ Knobs: base.TestingKnobs{ + DistSQL: &execinfra.TestingKnobs{ + // A low limit, to force many small scans such that we get progress + // reports. + TableReaderBatchBytesLimit: 1500, + }, Store: &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(_ context.Context, req roachpb.BatchRequest) *roachpb.Error { if req.IsSingleRequest() { @@ -605,11 +612,7 @@ func TestQueryProgress(t *testing.T) { if err != nil { t.Fatal(err) } - // Although we know we've scanned ~50% of what we'll scan, exactly when the - // meta makes its way back to the receiver vs when the progress is checked is - // non-deterministic so we could see 47% done or 53% done, etc. To avoid being - // flaky, we just make sure we see one of 4x% or 5x% - require.Regexp(t, `executing \([45]\d\.`, progress) + require.Regexp(t, `executing \(51\.20%\)`, progress) } // This test ensures that when in an explicit transaction, statement preparation diff --git a/pkg/sql/create_stats_test.go b/pkg/sql/create_stats_test.go index 9e917b89f86d..678963c69aa0 100644 --- a/pkg/sql/create_stats_test.go +++ b/pkg/sql/create_stats_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -39,9 +40,14 @@ func TestStatsWithLowTTL(t *testing.T) { // The test depends on reasonable timings, so don't run under race. skip.UnderRace(t) - // Set the KV batch size to 1 to avoid having to use a large number of rows. - defer row.TestingSetKVBatchSize(1)() - s, db, _ := serverutils.StartServer(t, base.TestServerArgs{}) + s, db, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + DistSQL: &execinfra.TestingKnobs{ + // Set the batch size small to avoid having to use a large number of rows. + TableReaderBatchBytesLimit: 100, + }, + }, + }) defer s.Stopper().Stop(context.Background()) r := sqlutils.MakeSQLRunner(db) @@ -51,7 +57,7 @@ func TestStatsWithLowTTL(t *testing.T) { USE test; CREATE TABLE t (k INT PRIMARY KEY, a INT, b INT); `) - const numRows = 5 + const numRows = 20 r.Exec(t, `INSERT INTO t SELECT k, 2*k, 3*k FROM generate_series(0, $1) AS g(k)`, numRows-1) pgURL, cleanupFunc := sqlutils.PGUrl(t, @@ -109,7 +115,7 @@ func TestStatsWithLowTTL(t *testing.T) { }() // Sleep 500ms after every scanned row (note that we set the KV batch size to - // 1 above), to simulate a long-running operation. + // a small limit above), to simulate a long-running operation. row.TestingInconsistentScanSleep = 500 * time.Millisecond defer func() { row.TestingInconsistentScanSleep = 0 }() diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 14b4d9bd937f..f2312f87f334 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1358,6 +1358,9 @@ func (dsp *DistSQLPlanner) planTableReaders( } tr.Parallelize = info.parallelize + if !tr.Parallelize { + tr.BatchBytesLimit = dsp.distSQLSrv.TestingKnobs.TableReaderBatchBytesLimit + } p.TotalEstimatedScannedRows += info.estimatedRowCount corePlacement[i].NodeID = sp.Node @@ -2136,6 +2139,7 @@ func (dsp *DistSQLPlanner) createPlanForLookupJoin( MaintainOrdering: len(n.reqOrdering) > 0, HasSystemColumns: n.table.containsSystemColumns, LeftJoinWithPairedJoiner: n.isSecondJoinInPairedJoiner, + LookupBatchBytesLimit: dsp.distSQLSrv.TestingKnobs.JoinReaderBatchBytesLimit, } joinReaderSpec.IndexIdx, err = getIndexIdx(n.table.index, n.table.desc) if err != nil { diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 128c24b952dc..59e2712fd620 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -210,6 +210,15 @@ type TestingKnobs struct { // Cannot be set together with ForceDiskSpill. MemoryLimitBytes int64 + // TableReaderBatchBytesLimit, if not 0, overrides the limit that the + // TableReader will set on the size of results it wants to get for individual + // requests. + TableReaderBatchBytesLimit int64 + // JoinReaderBatchBytesLimit, if not 0, overrides the limit that the + // joinReader will set on the size of results it wants to get for individual + // lookup requests. + JoinReaderBatchBytesLimit int64 + // DrainFast, if enabled, causes the server to not wait for any currently // running flows to complete or give a grace period of minFlowDrainWait // to incoming flows to register. diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 6b1f18893486..30dcb3a01937 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -589,6 +589,10 @@ type TableReaderSpec struct { // bound on the number of rows we can read, and when there is no limit or // limit hint. Parallelize bool `protobuf:"varint,12,opt,name=parallelize" json:"parallelize"` + // batch_bytes_limit, if non-zero, controls the TargetBytes limits that the + // TableReader will use for its scans. If zero, then the server-side default + // is used. If parallelize is set, this cannot be set. + BatchBytesLimit int64 `protobuf:"varint,17,opt,name=batch_bytes_limit,json=batchBytesLimit" json:"batch_bytes_limit"` // If non-zero, this enables inconsistent historical scanning where different // batches can be read with different timestamps. This is used for // long-running table statistics which may outlive the TTL. Using this setting @@ -951,6 +955,12 @@ type JoinReaderSpec struct { // OutputGroupContinuationForLeftRow is true, MaintainOrdering must also // be true. OutputGroupContinuationForLeftRow bool `protobuf:"varint,15,opt,name=output_group_continuation_for_left_row,json=outputGroupContinuationForLeftRow" json:"output_group_continuation_for_left_row"` + // lookup_batch_bytes_limit, if non-zero, controls the TargetBytes limits that + // the joiner will use for its lookups. If zero, then the server-side default + // is used. Note that, regardless of this setting, bytes limits are not always + // used for lookups - it depends on whether the joiner decides it wants + // DistSender-parallelism or not. + LookupBatchBytesLimit int64 `protobuf:"varint,18,opt,name=lookup_batch_bytes_limit,json=lookupBatchBytesLimit" json:"lookup_batch_bytes_limit"` } func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } @@ -1992,193 +2002,196 @@ func init() { } var fileDescriptor_90493aed97b6d28f = []byte{ - // 2975 bytes of a gzipped FileDescriptorProto + // 3016 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3a, 0x4b, 0x6f, 0x1b, 0xd7, - 0xb9, 0xe2, 0x4b, 0x22, 0x3f, 0x3e, 0x74, 0x7c, 0x6c, 0xc7, 0x8c, 0x92, 0x2b, 0xcb, 0xb4, 0x63, - 0xcb, 0x8e, 0x23, 0xdf, 0xe8, 0x5e, 0xe4, 0x22, 0xb9, 0x45, 0x51, 0x8a, 0x1c, 0xd2, 0x94, 0xa9, - 0x19, 0x7a, 0x48, 0x4a, 0x72, 0x02, 0xf4, 0x60, 0x44, 0x1e, 0x51, 0x13, 0x0f, 0x67, 0xa8, 0x99, - 0xa1, 0x25, 0xe5, 0x0f, 0x74, 0x55, 0xa0, 0x45, 0x37, 0x45, 0x16, 0x45, 0x36, 0x45, 0x81, 0xae, - 0xfb, 0x13, 0x8a, 0xc2, 0xcb, 0xa0, 0x8b, 0x36, 0xab, 0xa2, 0x71, 0x76, 0xfd, 0x03, 0x45, 0x77, - 0xc5, 0x79, 0xcc, 0x68, 0xa8, 0x88, 0x6e, 0x68, 0x3b, 0xf1, 0xc6, 0xd0, 0xf7, 0x3c, 0xdf, 0xf9, - 0xde, 0x67, 0x68, 0xb8, 0xe1, 0x1d, 0x5a, 0xf7, 0xe8, 0x31, 0xed, 0x99, 0xf6, 0xbe, 0x6b, 0x8c, - 0xf6, 0xee, 0x8d, 0x5c, 0xa7, 0x47, 0x3d, 0xcf, 0x71, 0x3d, 0xe2, 0x1d, 0x5a, 0x6b, 0x23, 0xd7, - 0xf1, 0x1d, 0x5c, 0xec, 0x39, 0xbd, 0xc7, 0xae, 0x63, 0xf4, 0x0e, 0xd6, 0x18, 0xb2, 0x6f, 0x7a, - 0xbe, 0x77, 0x68, 0xb9, 0x63, 0x7b, 0xe9, 0x3a, 0x93, 0xef, 0x19, 0xbe, 0x61, 0x39, 0x83, 0x7b, - 0x7d, 0xea, 0xf5, 0x46, 0x7b, 0xf7, 0x3c, 0xdf, 0x1d, 0xf7, 0xfc, 0xb1, 0x4b, 0xfb, 0x42, 0x7c, - 0xa9, 0x74, 0x0e, 0xd3, 0xa7, 0x8e, 0x69, 0x13, 0xff, 0x64, 0x44, 0x25, 0xcf, 0xca, 0x39, 0x3c, - 0x96, 0xd3, 0x7b, 0x6c, 0xda, 0x03, 0xc9, 0x71, 0x99, 0x71, 0x30, 0x09, 0x4f, 0xfc, 0x2b, 0xd1, - 0x4b, 0x67, 0x6f, 0xd0, 0x37, 0x7c, 0x43, 0xd2, 0xde, 0x79, 0xce, 0xed, 0xf6, 0x0c, 0x8f, 0x46, - 0xed, 0x33, 0xed, 0x27, 0xd4, 0xf5, 0x69, 0xff, 0x9e, 0x37, 0x32, 0x6c, 0x42, 0x8f, 0x47, 0x2e, - 0xf5, 0x3c, 0xd3, 0xb1, 0x25, 0xcf, 0xa5, 0x81, 0x33, 0x70, 0xf8, 0x9f, 0xf7, 0xd8, 0x5f, 0x02, - 0x5b, 0xfa, 0x55, 0x0c, 0x0a, 0xdb, 0x86, 0x35, 0xa6, 0x5e, 0xc5, 0x71, 0x69, 0x7b, 0x44, 0x7b, - 0xb8, 0x02, 0x0b, 0x3d, 0xc7, 0x1a, 0x0f, 0x6d, 0xaf, 0x18, 0x5b, 0x49, 0xac, 0x66, 0xd7, 0xaf, - 0xaf, 0x4d, 0xf3, 0xde, 0x5a, 0xd5, 0xf0, 0xc7, 0xc3, 0x86, 0xbd, 0xef, 0x6c, 0x24, 0x9f, 0xfe, - 0xed, 0xea, 0x9c, 0x1e, 0x48, 0xe2, 0xab, 0x90, 0xb6, 0xc7, 0x43, 0xe2, 0x3a, 0x47, 0x5e, 0x31, - 0xb1, 0x12, 0x5b, 0x4d, 0x06, 0x0c, 0xf6, 0x78, 0xa8, 0x3b, 0x47, 0x1e, 0x7e, 0x0b, 0x32, 0xae, - 0x71, 0x44, 0xf6, 0x4e, 0x7c, 0xea, 0x15, 0xe3, 0x2b, 0x89, 0xd5, 0x9c, 0x9e, 0x76, 0x8d, 0xa3, - 0x0d, 0x06, 0x97, 0xfe, 0x3c, 0x0f, 0x8b, 0x1d, 0x63, 0xcf, 0xa2, 0x3a, 0x35, 0xfa, 0xd4, 0xe5, - 0x66, 0x6d, 0x40, 0xca, 0x67, 0xa8, 0x62, 0x6c, 0x25, 0xb6, 0x9a, 0x5d, 0xbf, 0x79, 0xc6, 0x28, - 0xef, 0xd0, 0xe2, 0x0e, 0xe1, 0x62, 0x55, 0xea, 0xf5, 0x5c, 0x73, 0xe4, 0x3b, 0xae, 0x3c, 0x56, - 0x88, 0xe2, 0x6b, 0x90, 0x31, 0xed, 0x3e, 0x3d, 0x26, 0x66, 0xff, 0xb8, 0x18, 0x5f, 0x89, 0xad, - 0xe6, 0x25, 0x3d, 0xcd, 0xd1, 0x8d, 0xfe, 0x31, 0x5e, 0x86, 0x05, 0x97, 0x3e, 0xa1, 0xae, 0x47, - 0xb9, 0xdd, 0xe9, 0xc0, 0x6e, 0x89, 0xc4, 0x0a, 0xa4, 0x98, 0x7f, 0xbd, 0x62, 0x92, 0xfb, 0xe6, - 0xf6, 0x74, 0xdf, 0x4c, 0x5c, 0xc0, 0xb0, 0x03, 0x4b, 0xb8, 0x34, 0xbe, 0x0e, 0x60, 0x99, 0x43, - 0xd3, 0x27, 0x07, 0xa6, 0xed, 0x17, 0x53, 0x2b, 0xb1, 0xd5, 0x84, 0x64, 0xc8, 0x70, 0xfc, 0x7d, - 0xd3, 0xf6, 0x99, 0x13, 0x4d, 0x8f, 0xf4, 0x0e, 0x68, 0xef, 0x71, 0x71, 0x3e, 0x6a, 0x8c, 0xe9, - 0x55, 0x18, 0x12, 0xab, 0x00, 0x4f, 0x4c, 0xcf, 0xdc, 0x33, 0x2d, 0xd3, 0x3f, 0x29, 0x2e, 0xac, - 0xc4, 0x56, 0x0b, 0xeb, 0xab, 0xd3, 0x2d, 0x6a, 0xf7, 0x0c, 0x7b, 0x3b, 0xe4, 0x97, 0xca, 0x22, - 0x1a, 0xf0, 0x4d, 0xc8, 0x8e, 0x0c, 0xd7, 0xb0, 0x2c, 0x6a, 0x99, 0x9f, 0xd1, 0x62, 0x2e, 0x72, - 0x66, 0x94, 0x80, 0xff, 0x1f, 0xae, 0x0c, 0x8d, 0x63, 0xe2, 0x9b, 0x43, 0xea, 0xf9, 0xc6, 0x70, - 0x44, 0x8c, 0x01, 0x25, 0xb6, 0x61, 0x3b, 0x5e, 0x31, 0x13, 0x09, 0xf6, 0xa5, 0xa1, 0x71, 0xdc, - 0x09, 0x78, 0xca, 0x03, 0xaa, 0x32, 0x0e, 0xfc, 0x09, 0x20, 0x59, 0x17, 0xc4, 0xf3, 0x5d, 0x6a, - 0x0f, 0xfc, 0x83, 0x22, 0x70, 0xd3, 0xef, 0x4c, 0x89, 0x29, 0xb3, 0xbb, 0x29, 0x44, 0xda, 0x52, - 0x42, 0x9e, 0xb0, 0x68, 0x4d, 0xa2, 0xf1, 0x1e, 0x5c, 0x0c, 0x94, 0x1f, 0x19, 0xa6, 0x4f, 0x46, - 0x8e, 0x65, 0xf6, 0x4e, 0x8a, 0x59, 0xae, 0xff, 0xee, 0x7f, 0xd6, 0xbf, 0x63, 0x98, 0x7e, 0x8b, - 0xcb, 0xc8, 0x13, 0x2e, 0x58, 0x67, 0x09, 0x78, 0x1d, 0xf0, 0x81, 0xe1, 0x11, 0xef, 0xc4, 0xf3, - 0xe9, 0x90, 0x04, 0xb5, 0x52, 0x88, 0x38, 0x0b, 0x1d, 0x18, 0x5e, 0x9b, 0x93, 0x2b, 0xb2, 0x1e, - 0xde, 0x81, 0x82, 0x4d, 0x69, 0x9f, 0xf6, 0x43, 0xfe, 0xc5, 0x95, 0xc4, 0x6a, 0x5e, 0xcf, 0x0b, - 0x6c, 0xc0, 0xa6, 0x42, 0xe1, 0x89, 0xe9, 0xfa, 0x63, 0xc3, 0x92, 0x7c, 0x45, 0xc4, 0xb3, 0xfd, - 0xd6, 0x14, 0xcb, 0x85, 0xdc, 0x69, 0xba, 0xeb, 0x79, 0x29, 0x2e, 0x08, 0x9b, 0xc9, 0x74, 0x1a, - 0x65, 0x36, 0x93, 0xe9, 0x3c, 0x2a, 0x94, 0x74, 0xc8, 0xd5, 0x4c, 0xcb, 0xa7, 0x6e, 0x58, 0x50, - 0xf3, 0xfb, 0x1c, 0x96, 0x15, 0x75, 0x63, 0x7a, 0xe2, 0x28, 0x61, 0x33, 0x91, 0x17, 0x94, 0x92, - 0xa5, 0x7f, 0x25, 0xa0, 0xd8, 0x60, 0xa5, 0xd3, 0x7e, 0x6c, 0x8e, 0x5e, 0x53, 0xc5, 0x86, 0x15, - 0x99, 0x78, 0xa9, 0x8a, 0x9c, 0xac, 0xa5, 0xe4, 0x4b, 0xd7, 0x52, 0xa4, 0x91, 0xa4, 0xce, 0x6b, - 0x24, 0xe7, 0x95, 0xc1, 0xfc, 0xf7, 0x5c, 0x06, 0x0b, 0xaf, 0xb0, 0x0c, 0x4a, 0xbf, 0x4b, 0x43, - 0x61, 0xd3, 0x31, 0xed, 0x1f, 0x3e, 0xe2, 0x0f, 0x20, 0x6b, 0x39, 0xce, 0xe3, 0xf1, 0x88, 0x4f, - 0x39, 0x59, 0x22, 0xb3, 0xa4, 0x2f, 0x08, 0x71, 0x86, 0xc7, 0xbb, 0x80, 0x5d, 0x3a, 0x74, 0x7c, - 0x4a, 0xa2, 0x3a, 0x2f, 0xcc, 0xac, 0x13, 0x09, 0x2d, 0xcd, 0x53, 0xcd, 0xb7, 0xa1, 0x20, 0x55, - 0x06, 0x35, 0xcf, 0x32, 0x34, 0xbf, 0x11, 0x47, 0x31, 0x3d, 0x2f, 0x28, 0x41, 0xdd, 0x7f, 0x08, - 0x6f, 0x4c, 0xb2, 0x12, 0xc3, 0xa5, 0xe4, 0x31, 0x3d, 0x29, 0xa6, 0x23, 0xb9, 0x73, 0x71, 0x42, - 0xa8, 0xec, 0xd2, 0x07, 0xf4, 0x84, 0x8d, 0x6b, 0x47, 0x8c, 0x7b, 0x9e, 0xb4, 0x33, 0xd6, 0xb1, - 0x63, 0x73, 0x53, 0x3f, 0x84, 0x24, 0x5b, 0x49, 0x64, 0x02, 0x5e, 0x9d, 0x12, 0x37, 0x16, 0xed, - 0xce, 0xc9, 0x88, 0x4a, 0x61, 0x2e, 0xf2, 0xca, 0x67, 0xd0, 0x79, 0x75, 0x91, 0xf9, 0x9e, 0xeb, - 0x02, 0x5e, 0xe5, 0x78, 0x78, 0x1f, 0x2e, 0x0c, 0x0d, 0xd3, 0xf6, 0x0d, 0xd3, 0x26, 0x8e, 0xdb, - 0xa7, 0xae, 0x69, 0x0f, 0xf8, 0x00, 0x0a, 0xa7, 0x43, 0x40, 0xd6, 0x24, 0x75, 0xca, 0x44, 0xc9, - 0x3f, 0x77, 0xa2, 0x54, 0xe1, 0x6d, 0x8b, 0xee, 0xfb, 0x84, 0xef, 0xa1, 0x47, 0xa6, 0x7f, 0x40, - 0x46, 0x86, 0xe9, 0xd2, 0x3e, 0x47, 0x50, 0x77, 0x62, 0x1e, 0x15, 0x19, 0x27, 0x8b, 0xde, 0x8e, - 0xe9, 0x1f, 0xb4, 0x38, 0xdb, 0x26, 0xe7, 0xc2, 0x3b, 0x70, 0xd3, 0x19, 0xfb, 0xa3, 0xb1, 0x4f, - 0x06, 0xae, 0xc3, 0xd3, 0xcf, 0xf6, 0x4d, 0x7b, 0x6c, 0xf8, 0xa6, 0x63, 0x93, 0x7d, 0xc7, 0x25, - 0xfc, 0x0c, 0xd7, 0x39, 0x2a, 0x2e, 0x46, 0xf4, 0x5d, 0x13, 0x32, 0x75, 0x26, 0x52, 0x89, 0x48, - 0xd4, 0x1c, 0xb7, 0x49, 0xf7, 0x7d, 0xdd, 0x39, 0xda, 0x4c, 0xa6, 0x53, 0x68, 0x7e, 0x33, 0x99, - 0xce, 0xa1, 0x3c, 0x5b, 0x32, 0xa1, 0xed, 0xb8, 0xbe, 0xec, 0x12, 0x0f, 0x61, 0x51, 0x9e, 0x19, - 0xba, 0x47, 0xf4, 0x8b, 0xd2, 0xf4, 0xb4, 0x09, 0x5c, 0x25, 0x0d, 0x28, 0x08, 0x05, 0x51, 0x07, - 0x06, 0xba, 0xc8, 0xd0, 0xf0, 0x7b, 0x07, 0xc4, 0xa2, 0xf6, 0x44, 0xf7, 0x40, 0x01, 0x7d, 0x8b, - 0x91, 0x9b, 0xd4, 0x2e, 0xfd, 0x32, 0x0e, 0xb9, 0xaa, 0xe9, 0xf9, 0xa6, 0xdd, 0xf3, 0xb9, 0x5d, - 0xb7, 0x60, 0x91, 0x33, 0x45, 0x86, 0x74, 0x8c, 0x0f, 0xe9, 0x82, 0x44, 0x07, 0xae, 0xbf, 0x0d, - 0xa8, 0x2f, 0x05, 0x43, 0xce, 0x38, 0xe7, 0x5c, 0x0c, 0xf0, 0x01, 0xeb, 0x3a, 0x60, 0x7b, 0x6c, - 0x59, 0xa2, 0x9e, 0x03, 0xe2, 0xc4, 0x66, 0x89, 0x38, 0xbd, 0xec, 0xd2, 0xc0, 0x16, 0x7c, 0x13, - 0x72, 0xd4, 0x75, 0x1d, 0x97, 0x38, 0x36, 0xe9, 0x8f, 0x47, 0xbc, 0xac, 0x33, 0x41, 0xa5, 0x70, - 0x8a, 0x66, 0x57, 0xc7, 0xa3, 0xf3, 0xfc, 0x98, 0x7a, 0x39, 0x3f, 0x96, 0x10, 0x14, 0x34, 0xb7, - 0x6f, 0xda, 0x06, 0x2b, 0x45, 0xe6, 0x94, 0xd2, 0xaf, 0x13, 0x80, 0x3e, 0x36, 0x07, 0x9f, 0x19, - 0x03, 0x91, 0x31, 0xdc, 0x53, 0x55, 0x98, 0xe7, 0xcd, 0x3a, 0x78, 0x21, 0xcc, 0xd6, 0xe8, 0xa5, - 0x2c, 0xae, 0x01, 0xd0, 0xc3, 0x09, 0x07, 0x66, 0xd7, 0xaf, 0x4d, 0x37, 0x5d, 0xba, 0x34, 0x58, - 0x93, 0xe9, 0xe1, 0x69, 0x38, 0x0a, 0x62, 0x62, 0x38, 0xc2, 0xf4, 0x89, 0x3e, 0xcb, 0x29, 0xf2, - 0x4e, 0xde, 0xab, 0x69, 0x96, 0x0f, 0x20, 0xb7, 0x6f, 0x1e, 0xd3, 0x3e, 0x79, 0xc2, 0x1f, 0x4e, - 0xc5, 0x14, 0xb7, 0xfc, 0x39, 0x3d, 0x6f, 0xf2, 0x81, 0xa5, 0x67, 0xb9, 0xb4, 0x40, 0xbe, 0x44, - 0xe7, 0x2d, 0xfd, 0x25, 0x01, 0x8b, 0x5b, 0xd4, 0x1d, 0xd0, 0x48, 0x64, 0xb6, 0x20, 0xcf, 0x2b, - 0xf6, 0x85, 0x2b, 0x2b, 0xc7, 0xc4, 0xc3, 0xba, 0xd2, 0xa0, 0xe0, 0x9a, 0x83, 0x83, 0x88, 0xbe, - 0xf8, 0x8c, 0xfa, 0xf2, 0x5c, 0x3e, 0x54, 0x18, 0x09, 0x40, 0xea, 0x75, 0x4c, 0xab, 0xdb, 0x90, - 0x67, 0xf5, 0x46, 0xe8, 0xe1, 0xd8, 0x08, 0x07, 0x56, 0x50, 0x8a, 0x39, 0x46, 0x52, 0x24, 0x05, - 0x7f, 0x04, 0x57, 0xb8, 0x2b, 0x4f, 0x73, 0x74, 0xca, 0x50, 0xa6, 0xfb, 0xbe, 0x72, 0x38, 0x39, - 0x94, 0x7f, 0x04, 0x45, 0xe1, 0xb7, 0x73, 0x84, 0x33, 0x11, 0xe1, 0x4b, 0x9c, 0xeb, 0x8c, 0x74, - 0xe9, 0x1f, 0x71, 0x28, 0xdc, 0x37, 0xbc, 0x83, 0x48, 0x5c, 0xef, 0xc0, 0xe2, 0x19, 0x63, 0x44, - 0x6f, 0x92, 0xcb, 0x44, 0xd4, 0x04, 0x7c, 0x17, 0xd0, 0xd9, 0xc3, 0x45, 0x7b, 0xe2, 0xcc, 0x85, - 0xc9, 0x23, 0x5f, 0x7b, 0x44, 0x5e, 0x9b, 0x9b, 0x37, 0x93, 0xe9, 0x05, 0x94, 0x2e, 0x7d, 0x9e, - 0x04, 0xdc, 0x90, 0x9f, 0x4e, 0x22, 0x0e, 0xff, 0x81, 0x56, 0x59, 0x0d, 0xf2, 0xc1, 0x77, 0x9b, - 0x17, 0x6d, 0x4b, 0xb9, 0x40, 0x01, 0x8f, 0xc4, 0xeb, 0x0e, 0xe7, 0xb9, 0xdb, 0xcf, 0xc2, 0x73, - 0xb7, 0x9f, 0xef, 0xbe, 0x83, 0xa4, 0x67, 0xda, 0x41, 0x58, 0x6a, 0x8d, 0x5c, 0xba, 0x6f, 0x1e, - 0x87, 0xe5, 0x1e, 0xd6, 0x43, 0x26, 0xac, 0x87, 0xcb, 0x82, 0x25, 0x28, 0x7b, 0x99, 0x22, 0x9b, - 0xc9, 0x74, 0x02, 0x25, 0x4b, 0x9f, 0x27, 0xe0, 0x52, 0x90, 0x1c, 0x13, 0x8f, 0xe7, 0x35, 0x40, - 0x61, 0xdc, 0x7a, 0x8e, 0xc5, 0x23, 0x1c, 0x8b, 0x44, 0xb8, 0x10, 0x50, 0x2b, 0x8e, 0xc5, 0xe2, - 0xbc, 0x7d, 0x36, 0xce, 0xa2, 0x8f, 0xbe, 0x7b, 0xc6, 0xb5, 0x01, 0xcf, 0x1a, 0x7b, 0x9f, 0x9e, - 0x86, 0xa7, 0xe5, 0x3a, 0xbe, 0x73, 0x6e, 0xb8, 0x3f, 0x85, 0x0b, 0x23, 0x97, 0x92, 0x7d, 0x69, - 0x1b, 0xf1, 0x46, 0xb4, 0xc7, 0xc3, 0x96, 0x5d, 0xff, 0xf1, 0xf4, 0xc0, 0x9f, 0x77, 0xa5, 0xb5, - 0x96, 0x4b, 0xa3, 0xb0, 0xbe, 0x38, 0x9a, 0x44, 0x2c, 0xfd, 0x3c, 0x06, 0x8b, 0x67, 0x98, 0xf0, - 0x26, 0xc0, 0xe9, 0x97, 0xc6, 0x17, 0xf8, 0x90, 0x10, 0x91, 0xc6, 0x6b, 0x32, 0xeb, 0x84, 0x6b, - 0x96, 0xce, 0x66, 0x1d, 0x1d, 0xae, 0x89, 0xcf, 0xa6, 0x1d, 0x39, 0xff, 0xfe, 0x04, 0x50, 0x28, - 0x0f, 0x06, 0x2e, 0x1d, 0x18, 0xbe, 0x23, 0xcc, 0xb9, 0x06, 0x10, 0xe4, 0x90, 0x15, 0x6d, 0x7a, - 0x99, 0x81, 0x48, 0x13, 0xcb, 0xc3, 0x3f, 0x85, 0x9c, 0x21, 0x85, 0x4c, 0x27, 0xfc, 0x6a, 0xf0, - 0xbf, 0xd3, 0x6d, 0x9e, 0x3c, 0x22, 0x04, 0x23, 0x05, 0x18, 0xd5, 0x87, 0xff, 0x5b, 0xae, 0xa2, - 0xb4, 0x4f, 0x22, 0xa6, 0x24, 0x43, 0x53, 0x90, 0xa4, 0xd6, 0x43, 0x8b, 0xea, 0xf2, 0xde, 0x29, - 0x5e, 0x6d, 0xef, 0x7d, 0x67, 0x4b, 0xbe, 0x55, 0x7b, 0xe7, 0x2c, 0x84, 0xf3, 0x2f, 0xb7, 0x10, - 0x2e, 0xfd, 0x2c, 0x0e, 0xd9, 0xc8, 0x8d, 0x99, 0xad, 0xfb, 0x63, 0xbb, 0xc7, 0x23, 0x3d, 0x8b, - 0xad, 0xb5, 0xb1, 0xdd, 0x0b, 0x6c, 0x65, 0x0a, 0xf0, 0x0a, 0xa4, 0xc3, 0x75, 0x38, 0x1e, 0x29, - 0xeb, 0x10, 0x8b, 0xaf, 0xf0, 0xef, 0xd0, 0xbc, 0xb2, 0x52, 0x7c, 0xb9, 0x9e, 0xef, 0x89, 0x5a, - 0xba, 0x01, 0x05, 0x91, 0xef, 0x61, 0xe5, 0xb1, 0xa6, 0x99, 0xd7, 0x73, 0x02, 0x2b, 0x2b, 0xee, - 0x3e, 0x64, 0x0c, 0x77, 0x30, 0x1e, 0x52, 0xdb, 0xf7, 0x8a, 0xf3, 0x3c, 0xc8, 0xb3, 0x24, 0xe6, - 0xa9, 0xb0, 0x6c, 0x05, 0xbf, 0x4f, 0x41, 0x92, 0xdd, 0x02, 0x23, 0xc8, 0x95, 0xd5, 0x47, 0x44, - 0xd5, 0x3a, 0x44, 0xed, 0x36, 0x9b, 0x68, 0x0e, 0x2f, 0x40, 0xa2, 0xbc, 0x5d, 0x47, 0x31, 0x9c, - 0x83, 0xf4, 0x86, 0xa6, 0x35, 0x49, 0x59, 0xad, 0xa2, 0x38, 0xce, 0xc2, 0x02, 0x87, 0x34, 0x1d, - 0x25, 0x70, 0x01, 0xa0, 0xa2, 0xa9, 0x95, 0x72, 0x87, 0x94, 0xeb, 0x75, 0x94, 0xc4, 0x19, 0x48, - 0x55, 0xb4, 0xae, 0xda, 0x41, 0x29, 0x26, 0xbe, 0x55, 0xde, 0x45, 0x0b, 0xfc, 0x8f, 0x86, 0x8a, - 0xd2, 0x18, 0x60, 0xbe, 0xdd, 0xa9, 0x56, 0x95, 0x6d, 0x94, 0x61, 0xc8, 0x76, 0x77, 0x0b, 0x01, - 0x53, 0xd7, 0xee, 0x6e, 0x91, 0x86, 0xda, 0x41, 0x59, 0x76, 0xd2, 0x76, 0x59, 0x6f, 0x94, 0xd5, - 0x8a, 0x82, 0x72, 0x8c, 0xb4, 0xab, 0xe9, 0x5c, 0x73, 0x5e, 0x9c, 0xd4, 0x55, 0x3b, 0x44, 0xd7, - 0x76, 0xda, 0xa8, 0xc0, 0xe5, 0x1e, 0xea, 0xd5, 0x46, 0xad, 0x86, 0x16, 0x31, 0x86, 0x42, 0xad, - 0xa1, 0x96, 0x9b, 0x24, 0x94, 0x46, 0xec, 0x42, 0x02, 0x27, 0xcf, 0xbc, 0x80, 0xf3, 0x90, 0x29, - 0xeb, 0x7a, 0xf9, 0x11, 0xd7, 0x88, 0xd9, 0x61, 0x9b, 0x6d, 0x4d, 0xe5, 0xd0, 0x45, 0x46, 0x64, - 0xd0, 0x06, 0x07, 0x2f, 0xb1, 0xe3, 0xda, 0x1d, 0xbd, 0xa1, 0xd6, 0x39, 0x7c, 0x99, 0xdf, 0xba, - 0xd1, 0xe1, 0x2e, 0x78, 0x83, 0x5d, 0x84, 0x01, 0x9a, 0x8e, 0xae, 0xe0, 0x34, 0x24, 0x2b, 0x9a, - 0xae, 0xa3, 0x22, 0x2e, 0xc2, 0xa5, 0x96, 0xa2, 0x57, 0x14, 0xb5, 0xd3, 0x68, 0x2a, 0xa4, 0xda, - 0x68, 0x57, 0x48, 0x63, 0xab, 0xd5, 0x44, 0x6f, 0x9e, 0xa1, 0x54, 0x34, 0xb5, 0x23, 0x28, 0x4b, - 0xf8, 0x22, 0x2c, 0x72, 0x1b, 0xb4, 0x8d, 0x4d, 0xa5, 0x22, 0x9c, 0xf8, 0x16, 0xbe, 0x04, 0x48, - 0x98, 0x12, 0xc1, 0xbe, 0xcd, 0x2c, 0xd8, 0x2e, 0xeb, 0xa4, 0xa5, 0xb5, 0xd0, 0x7f, 0x09, 0xf3, - 0xd8, 0xb5, 0x38, 0xbc, 0x8c, 0x17, 0x21, 0xdb, 0xee, 0x90, 0xad, 0xf2, 0x03, 0xa5, 0xd9, 0x50, - 0x15, 0x74, 0x95, 0x5d, 0xa7, 0xdd, 0x21, 0xca, 0x6e, 0x47, 0x51, 0x3b, 0x68, 0x85, 0xdd, 0xb5, - 0xdd, 0x21, 0x5d, 0xb5, 0xa1, 0xa9, 0xe8, 0x9a, 0x90, 0x26, 0x15, 0xad, 0xd9, 0x54, 0x2a, 0x1d, - 0x54, 0x62, 0xcc, 0x15, 0x2d, 0x50, 0x7e, 0x5d, 0xb8, 0x9a, 0x81, 0xed, 0xf2, 0x56, 0x0b, 0xdd, - 0x60, 0xde, 0xd5, 0x95, 0xba, 0xce, 0x62, 0xc4, 0x6e, 0xd1, 0xea, 0xa0, 0x77, 0x98, 0x35, 0x1c, - 0xa7, 0xaf, 0xa3, 0x9b, 0x4c, 0x80, 0x03, 0xed, 0xa6, 0xd6, 0x52, 0xd0, 0x2d, 0x76, 0x9a, 0x80, - 0x77, 0x77, 0xd1, 0xea, 0x29, 0xf4, 0xe8, 0x11, 0xba, 0x1d, 0xa1, 0x3d, 0x42, 0x77, 0x42, 0x49, - 0x91, 0x34, 0xef, 0x32, 0x4b, 0x38, 0x5c, 0xde, 0xae, 0xef, 0xa2, 0xbb, 0x51, 0xf0, 0x11, 0x7a, - 0xaf, 0x74, 0x17, 0x92, 0xac, 0x3b, 0x30, 0x9f, 0x97, 0xbb, 0x1d, 0x0d, 0xcd, 0xf1, 0x94, 0xaa, - 0x94, 0x9b, 0x65, 0x1d, 0xc5, 0x98, 0x2e, 0x55, 0x53, 0x89, 0x84, 0xe3, 0xa5, 0x3f, 0xc6, 0xa0, - 0xd0, 0x72, 0x9d, 0x4f, 0x69, 0xcf, 0x6f, 0x53, 0xf1, 0x16, 0xfe, 0x09, 0xa4, 0x58, 0x67, 0x0e, - 0x1e, 0x78, 0xb3, 0x54, 0x8e, 0x10, 0xc4, 0x75, 0xb8, 0x30, 0xa0, 0x36, 0x75, 0x0d, 0x3f, 0xf2, - 0x9e, 0x16, 0x8f, 0xbc, 0xe7, 0xb5, 0x76, 0x14, 0x0a, 0x05, 0x0b, 0xea, 0x2d, 0x40, 0xf6, 0x98, - 0x7f, 0x15, 0xf1, 0xc8, 0x88, 0xba, 0x64, 0x40, 0x6d, 0xf1, 0xc0, 0xd3, 0xf3, 0xf6, 0x78, 0xc8, - 0x3a, 0x68, 0x8b, 0xba, 0x75, 0x6a, 0x97, 0xbe, 0xce, 0x43, 0x6e, 0xc7, 0xb4, 0xfb, 0xce, 0x91, - 0x1c, 0x4e, 0x2b, 0xfc, 0xe7, 0x0c, 0xdf, 0xe4, 0xbd, 0xfa, 0x44, 0x3e, 0xe6, 0xa3, 0x28, 0xdc, - 0x86, 0xcc, 0x11, 0x97, 0xa8, 0x85, 0xc6, 0xdd, 0x9b, 0x7e, 0xd5, 0xa8, 0x72, 0x09, 0xd4, 0xc2, - 0x7e, 0x11, 0xea, 0x59, 0xfa, 0x43, 0x4c, 0x76, 0x8a, 0x36, 0xe4, 0x83, 0xd1, 0x40, 0x6b, 0x2f, - 0xda, 0x35, 0xf5, 0x49, 0x1d, 0xf8, 0x21, 0x80, 0x3c, 0x8a, 0x69, 0x8c, 0x73, 0x8d, 0xef, 0xcf, - 0x66, 0x33, 0xd3, 0x1a, 0x51, 0xf2, 0x51, 0xf2, 0xe9, 0x17, 0x57, 0x63, 0x4b, 0x5f, 0x2c, 0x40, - 0xaa, 0xe6, 0x1a, 0x43, 0x8a, 0x1f, 0x40, 0x72, 0xe8, 0xf4, 0xa9, 0x34, 0xf7, 0xbb, 0x2a, 0xe7, - 0xb2, 0x6b, 0x5b, 0x4e, 0x3f, 0x1c, 0x4a, 0x4c, 0x09, 0x7e, 0x08, 0xf3, 0x7b, 0xce, 0xd8, 0xee, - 0x7b, 0x72, 0xae, 0xff, 0xcf, 0x4c, 0xea, 0x36, 0xb8, 0x68, 0xb0, 0x9e, 0x0a, 0x45, 0xf8, 0x63, - 0xc8, 0xd0, 0xe3, 0x9e, 0x35, 0xe6, 0x3b, 0x47, 0x82, 0x1b, 0xf9, 0xc1, 0x4c, 0x5a, 0x95, 0x40, - 0x3a, 0xfc, 0x98, 0x10, 0x20, 0x96, 0xfe, 0x19, 0x83, 0x14, 0x3f, 0x94, 0x9d, 0xc2, 0xcf, 0x63, - 0x85, 0x24, 0x5d, 0xf1, 0xc1, 0xec, 0xb6, 0x47, 0x86, 0xf4, 0xa9, 0x3a, 0x7c, 0x1d, 0xc0, 0xb4, - 0x7d, 0xe2, 0xec, 0xef, 0x7b, 0x54, 0xcc, 0xbf, 0xe0, 0x37, 0xb3, 0x8c, 0x69, 0xfb, 0x1a, 0x47, - 0xe3, 0x6b, 0x90, 0x63, 0x55, 0xd1, 0x0f, 0xd8, 0xd8, 0x4d, 0x73, 0x7a, 0x96, 0xe3, 0x24, 0xcb, - 0x26, 0x64, 0x05, 0x91, 0xff, 0x12, 0x2d, 0x1f, 0x0f, 0x33, 0xfc, 0x5e, 0x0b, 0x42, 0x9a, 0xd9, - 0xb4, 0xf4, 0x9b, 0x18, 0xcc, 0x0b, 0x77, 0x63, 0x15, 0x52, 0x9e, 0x6f, 0xb8, 0xbe, 0x5c, 0xe8, - 0xd6, 0x67, 0xbf, 0x76, 0xf8, 0xdb, 0x0a, 0x53, 0x83, 0xab, 0x90, 0xa0, 0x76, 0x5f, 0x26, 0xc0, - 0x0b, 0x68, 0xd3, 0x99, 0x78, 0xe9, 0x16, 0x24, 0x59, 0x76, 0xb1, 0xd1, 0xa9, 0x97, 0xd5, 0xba, - 0x82, 0xe6, 0x58, 0x7f, 0xe3, 0x53, 0x2e, 0xc6, 0xfa, 0x5b, 0x5d, 0xd7, 0xba, 0xad, 0x36, 0x8a, - 0x97, 0x3e, 0x83, 0x4c, 0xe8, 0x7b, 0x7c, 0x05, 0x2e, 0x76, 0xd5, 0x0d, 0xad, 0xab, 0x56, 0x95, - 0x2a, 0x69, 0xe9, 0x4a, 0x45, 0xa9, 0x36, 0xd4, 0x3a, 0x9a, 0x9b, 0x24, 0xd4, 0xb4, 0x66, 0x53, - 0xdb, 0x61, 0x84, 0x18, 0x9b, 0x2a, 0x5a, 0xad, 0xd6, 0x56, 0x3a, 0x11, 0xf6, 0x78, 0x04, 0x7b, - 0xca, 0x9b, 0x60, 0xe3, 0xa4, 0xd2, 0xd5, 0x75, 0x45, 0x8c, 0x5b, 0x94, 0x2c, 0x7d, 0x02, 0x99, - 0x30, 0xbb, 0xd8, 0x64, 0x55, 0x35, 0xa2, 0xec, 0x56, 0x9a, 0xdd, 0x36, 0x1b, 0x28, 0xfc, 0x50, - 0x0e, 0x56, 0x15, 0x12, 0x95, 0x8b, 0xe1, 0x0b, 0x90, 0x0f, 0x08, 0xfc, 0x1e, 0x28, 0xce, 0xa4, - 0x03, 0x54, 0xa7, 0xa1, 0xb4, 0x51, 0x62, 0xe9, 0xaf, 0x71, 0x48, 0x07, 0x7d, 0x07, 0x2b, 0x91, - 0x55, 0xec, 0xdb, 0x2f, 0x89, 0xa9, 0x5e, 0x3d, 0xbb, 0x88, 0x2d, 0x41, 0xda, 0x70, 0x07, 0x5e, - 0xa3, 0x7f, 0xec, 0x15, 0x17, 0x78, 0x87, 0x0c, 0x61, 0x5c, 0x85, 0x74, 0xb8, 0x49, 0x26, 0x67, - 0xdc, 0x24, 0x43, 0x49, 0xf6, 0x94, 0xde, 0x67, 0xb1, 0x94, 0x0f, 0xd2, 0xbb, 0xb3, 0xc4, 0x5f, - 0x17, 0xa2, 0x78, 0x15, 0x26, 0xb6, 0x3b, 0xbe, 0xd7, 0xa6, 0x82, 0xfd, 0x7b, 0x62, 0xef, 0x5b, - 0x85, 0x9c, 0xd8, 0x61, 0x25, 0x67, 0x3a, 0xf2, 0x2a, 0x9b, 0xa0, 0x6c, 0x26, 0xd3, 0x71, 0x94, - 0x90, 0xdb, 0xdd, 0x6f, 0x63, 0x00, 0xa7, 0xdd, 0x91, 0x4f, 0x5b, 0x6d, 0x87, 0xa8, 0xdd, 0xad, - 0x0d, 0x45, 0x97, 0x79, 0x56, 0x56, 0x1f, 0x88, 0xd9, 0x59, 0x55, 0xd4, 0xb6, 0x42, 0x38, 0xcc, - 0x83, 0x24, 0x37, 0x16, 0x81, 0x49, 0xf0, 0x1d, 0xa1, 0xbb, 0xc5, 0xf7, 0x9a, 0x8e, 0x58, 0xf4, - 0xf8, 0x36, 0x23, 0x16, 0xbd, 0x66, 0xb9, 0x8e, 0xe6, 0x99, 0xba, 0xa6, 0x52, 0xae, 0xa2, 0x05, - 0x96, 0x3f, 0xb5, 0x86, 0xde, 0xee, 0x90, 0xed, 0x72, 0xb3, 0xab, 0xa0, 0x34, 0xd3, 0xdf, 0x2c, - 0x87, 0x70, 0x86, 0x69, 0x53, 0x3b, 0xf7, 0x25, 0x08, 0x77, 0xfe, 0x0f, 0x0a, 0x93, 0xbf, 0xa0, - 0xb0, 0xc4, 0x6f, 0x75, 0x37, 0x9a, 0x8d, 0x0a, 0x9a, 0xc3, 0x6f, 0xc2, 0x65, 0xf1, 0x37, 0x5b, - 0xbf, 0xf8, 0x86, 0x2a, 0x49, 0xb1, 0x8d, 0xf7, 0x9e, 0x7e, 0xbd, 0x3c, 0xf7, 0xf4, 0xd9, 0x72, - 0xec, 0xcb, 0x67, 0xcb, 0xb1, 0xaf, 0x9e, 0x2d, 0xc7, 0xfe, 0xfe, 0x6c, 0x39, 0xf6, 0x8b, 0x6f, - 0x96, 0xe7, 0xbe, 0xfc, 0x66, 0x79, 0xee, 0xab, 0x6f, 0x96, 0xe7, 0x3e, 0xce, 0x46, 0xfe, 0x9f, - 0xc9, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x70, 0x1f, 0x46, 0x09, 0x57, 0x23, 0x00, 0x00, + 0xb9, 0xe2, 0x4b, 0x22, 0x3f, 0x3e, 0x74, 0x74, 0x6c, 0xc7, 0x8c, 0x92, 0x2b, 0xcb, 0xb4, 0x63, + 0xcb, 0x8e, 0x23, 0x27, 0xba, 0x17, 0xb9, 0x48, 0xee, 0x03, 0x97, 0x22, 0x87, 0x34, 0x65, 0x6a, + 0x86, 0x1e, 0x92, 0xb2, 0x9c, 0x00, 0xf7, 0x60, 0x44, 0x1e, 0x51, 0x13, 0x0f, 0x67, 0xa8, 0x99, + 0xa1, 0x25, 0x65, 0x5d, 0xa0, 0xab, 0x02, 0x2d, 0xba, 0x29, 0xb2, 0x28, 0xb2, 0xe9, 0xa6, 0xeb, + 0xfe, 0x84, 0xa2, 0xf0, 0x32, 0xab, 0x36, 0xab, 0xa2, 0x71, 0x80, 0x2e, 0xfa, 0x07, 0x8a, 0xee, + 0x8a, 0xf3, 0x98, 0xd1, 0x50, 0x11, 0xdd, 0xd0, 0x76, 0xe2, 0x8d, 0xa1, 0xef, 0x79, 0xbe, 0xf3, + 0xbd, 0xcf, 0xd0, 0x70, 0xdd, 0x3b, 0xb4, 0xee, 0xd2, 0x63, 0xda, 0x33, 0xed, 0x7d, 0xd7, 0x18, + 0xed, 0xdd, 0x1d, 0xb9, 0x4e, 0x8f, 0x7a, 0x9e, 0xe3, 0x7a, 0xc4, 0x3b, 0xb4, 0xd6, 0x47, 0xae, + 0xe3, 0x3b, 0xb8, 0xd8, 0x73, 0x7a, 0x8f, 0x5d, 0xc7, 0xe8, 0x1d, 0xac, 0x33, 0x64, 0xdf, 0xf4, + 0x7c, 0xef, 0xd0, 0x72, 0xc7, 0xf6, 0xf2, 0x35, 0x26, 0xdf, 0x33, 0x7c, 0xc3, 0x72, 0x06, 0x77, + 0xfb, 0xd4, 0xeb, 0x8d, 0xf6, 0xee, 0x7a, 0xbe, 0x3b, 0xee, 0xf9, 0x63, 0x97, 0xf6, 0x85, 0xf8, + 0x72, 0xe9, 0x1c, 0xa6, 0xcf, 0x1c, 0xd3, 0x26, 0xfe, 0xc9, 0x88, 0x4a, 0x9e, 0xd5, 0x73, 0x78, + 0x2c, 0xa7, 0xf7, 0xd8, 0xb4, 0x07, 0x92, 0xe3, 0x12, 0xe3, 0x60, 0x12, 0x9e, 0xf8, 0x57, 0xa2, + 0x97, 0xcf, 0xde, 0xa0, 0x6f, 0xf8, 0x86, 0xa4, 0xbd, 0xf3, 0x9c, 0xdb, 0xed, 0x19, 0x1e, 0x8d, + 0xda, 0x67, 0xda, 0x4f, 0xa8, 0xeb, 0xd3, 0xfe, 0x5d, 0x6f, 0x64, 0xd8, 0x84, 0x1e, 0x8f, 0x5c, + 0xea, 0x79, 0xa6, 0x63, 0x4b, 0x9e, 0x8b, 0x03, 0x67, 0xe0, 0xf0, 0x3f, 0xef, 0xb2, 0xbf, 0x04, + 0xb6, 0xf4, 0xcb, 0x18, 0x14, 0x76, 0x0c, 0x6b, 0x4c, 0xbd, 0x8a, 0xe3, 0xd2, 0xf6, 0x88, 0xf6, + 0x70, 0x05, 0x16, 0x7a, 0x8e, 0x35, 0x1e, 0xda, 0x5e, 0x31, 0xb6, 0x9a, 0x58, 0xcb, 0x6e, 0x5c, + 0x5b, 0x9f, 0xe6, 0xbd, 0xf5, 0xaa, 0xe1, 0x8f, 0x87, 0x0d, 0x7b, 0xdf, 0xd9, 0x4c, 0x3e, 0xfd, + 0xf3, 0x95, 0x39, 0x3d, 0x90, 0xc4, 0x57, 0x20, 0x6d, 0x8f, 0x87, 0xc4, 0x75, 0x8e, 0xbc, 0x62, + 0x62, 0x35, 0xb6, 0x96, 0x0c, 0x18, 0xec, 0xf1, 0x50, 0x77, 0x8e, 0x3c, 0xfc, 0x16, 0x64, 0x5c, + 0xe3, 0x88, 0xec, 0x9d, 0xf8, 0xd4, 0x2b, 0xc6, 0x57, 0x13, 0x6b, 0x39, 0x3d, 0xed, 0x1a, 0x47, + 0x9b, 0x0c, 0x2e, 0xfd, 0x64, 0x01, 0x16, 0x3b, 0xc6, 0x9e, 0x45, 0x75, 0x6a, 0xf4, 0xa9, 0xcb, + 0xcd, 0xda, 0x84, 0x94, 0xcf, 0x50, 0xc5, 0xd8, 0x6a, 0x6c, 0x2d, 0xbb, 0x71, 0xe3, 0x8c, 0x51, + 0xde, 0xa1, 0xc5, 0x1d, 0xc2, 0xc5, 0xaa, 0xd4, 0xeb, 0xb9, 0xe6, 0xc8, 0x77, 0x5c, 0x79, 0xac, + 0x10, 0xc5, 0x57, 0x21, 0x63, 0xda, 0x7d, 0x7a, 0x4c, 0xcc, 0xfe, 0x71, 0x31, 0xbe, 0x1a, 0x5b, + 0xcb, 0x4b, 0x7a, 0x9a, 0xa3, 0x1b, 0xfd, 0x63, 0xbc, 0x02, 0x0b, 0x2e, 0x7d, 0x42, 0x5d, 0x8f, + 0x72, 0xbb, 0xd3, 0x81, 0xdd, 0x12, 0x89, 0x15, 0x48, 0x31, 0xff, 0x7a, 0xc5, 0x24, 0xf7, 0xcd, + 0xad, 0xe9, 0xbe, 0x99, 0xb8, 0x80, 0x61, 0x07, 0x96, 0x70, 0x69, 0x7c, 0x0d, 0xc0, 0x32, 0x87, + 0xa6, 0x4f, 0x0e, 0x4c, 0xdb, 0x2f, 0xa6, 0x56, 0x63, 0x6b, 0x09, 0xc9, 0x90, 0xe1, 0xf8, 0x7b, + 0xa6, 0xed, 0x33, 0x27, 0x9a, 0x1e, 0xe9, 0x1d, 0xd0, 0xde, 0xe3, 0xe2, 0x7c, 0xd4, 0x18, 0xd3, + 0xab, 0x30, 0x24, 0x56, 0x01, 0x9e, 0x98, 0x9e, 0xb9, 0x67, 0x5a, 0xa6, 0x7f, 0x52, 0x5c, 0x58, + 0x8d, 0xad, 0x15, 0x36, 0xd6, 0xa6, 0x5b, 0xd4, 0xee, 0x19, 0xf6, 0x4e, 0xc8, 0x2f, 0x95, 0x45, + 0x34, 0xe0, 0x1b, 0x90, 0x1d, 0x19, 0xae, 0x61, 0x59, 0xd4, 0x32, 0x3f, 0xa7, 0xc5, 0x5c, 0xe4, + 0xcc, 0x28, 0x01, 0xbf, 0x0f, 0x4b, 0x7b, 0x86, 0xdf, 0x3b, 0x10, 0xe1, 0x23, 0xdc, 0xe2, 0xe2, + 0x52, 0xe4, 0x12, 0x8b, 0x9c, 0xcc, 0x83, 0xd9, 0x64, 0x44, 0xfc, 0x5f, 0x70, 0x79, 0x68, 0x1c, + 0x13, 0xdf, 0x1c, 0x52, 0xcf, 0x37, 0x86, 0x23, 0x62, 0x0c, 0x28, 0xb1, 0x0d, 0xdb, 0xf1, 0x8a, + 0x99, 0x48, 0x7a, 0x5c, 0x1c, 0x1a, 0xc7, 0x9d, 0x80, 0xa7, 0x3c, 0xa0, 0x2a, 0xe3, 0xc0, 0x9f, + 0x02, 0x92, 0x95, 0x44, 0x3c, 0xdf, 0xa5, 0xf6, 0xc0, 0x3f, 0x28, 0x02, 0xbf, 0xec, 0xed, 0x29, + 0x59, 0xc0, 0x6e, 0xda, 0x14, 0x22, 0x6d, 0x29, 0x11, 0x58, 0x66, 0x4d, 0xa2, 0xf1, 0x1e, 0x5c, + 0x08, 0x94, 0x1f, 0x19, 0xa6, 0x4f, 0x46, 0x8e, 0x65, 0xf6, 0x4e, 0x8a, 0x59, 0xae, 0xff, 0xce, + 0xbf, 0xd6, 0xff, 0xd0, 0x30, 0xfd, 0x16, 0x97, 0x91, 0x27, 0x2c, 0x59, 0x67, 0x09, 0x78, 0x03, + 0xf0, 0x81, 0xe1, 0x11, 0xef, 0xc4, 0xf3, 0xe9, 0x90, 0x04, 0xd5, 0x55, 0x88, 0xb8, 0x17, 0x1d, + 0x18, 0x5e, 0x9b, 0x93, 0x2b, 0xb2, 0x82, 0xde, 0x81, 0x82, 0x4d, 0x69, 0x9f, 0xf6, 0x43, 0xfe, + 0xc5, 0xd5, 0xc4, 0x5a, 0x5e, 0xcf, 0x0b, 0x6c, 0xc0, 0xa6, 0x42, 0xe1, 0x89, 0xe9, 0xfa, 0x63, + 0xc3, 0x92, 0x7c, 0x45, 0xc4, 0xeb, 0xe3, 0xe6, 0x14, 0xcb, 0x85, 0xdc, 0x69, 0x81, 0xe8, 0x79, + 0x29, 0x2e, 0x08, 0x5b, 0xc9, 0x74, 0x1a, 0x65, 0xb6, 0x92, 0xe9, 0x3c, 0x2a, 0x94, 0x74, 0xc8, + 0xd5, 0x4c, 0xcb, 0xa7, 0x6e, 0x58, 0x82, 0xf3, 0xfb, 0x1c, 0x96, 0x35, 0x78, 0x7d, 0x7a, 0xaa, + 0x29, 0x61, 0xfb, 0x91, 0x17, 0x94, 0x92, 0xa5, 0x7f, 0x24, 0xa0, 0xd8, 0x60, 0xc5, 0xd6, 0x7e, + 0x6c, 0x8e, 0x5e, 0x53, 0x8d, 0x87, 0x35, 0x9c, 0x78, 0xa9, 0x1a, 0x9e, 0xac, 0xbe, 0xe4, 0x4b, + 0x57, 0x5f, 0xa4, 0xf5, 0xa4, 0xce, 0x6b, 0x3d, 0xe7, 0x95, 0xc1, 0xfc, 0x0f, 0x5c, 0x06, 0x0b, + 0xaf, 0xb0, 0x0c, 0x4a, 0x7f, 0x4d, 0x43, 0x61, 0xcb, 0x31, 0xed, 0x1f, 0x3f, 0xe2, 0xf7, 0x21, + 0x6b, 0x39, 0xce, 0xe3, 0xf1, 0x88, 0xcf, 0x45, 0x59, 0x22, 0xb3, 0xa4, 0x2f, 0x08, 0x71, 0x86, + 0xc7, 0xbb, 0x80, 0x5d, 0x3a, 0x74, 0x7c, 0x4a, 0xa2, 0x3a, 0x97, 0x66, 0xd6, 0x89, 0x84, 0x96, + 0xe6, 0xa9, 0xe6, 0x5b, 0x50, 0x90, 0x2a, 0x83, 0x9a, 0x67, 0x19, 0x9a, 0xdf, 0x8c, 0xa3, 0x98, + 0x9e, 0x17, 0x94, 0xa0, 0xee, 0x3f, 0x82, 0x37, 0x26, 0x59, 0x89, 0xe1, 0x52, 0xf2, 0x98, 0x9e, + 0x14, 0xd3, 0x91, 0xdc, 0xb9, 0x30, 0x21, 0x54, 0x76, 0xe9, 0x7d, 0x7a, 0xc2, 0x06, 0xbc, 0x23, + 0x16, 0x04, 0x9e, 0xb4, 0x33, 0xd6, 0xb1, 0x63, 0x73, 0x53, 0x3f, 0x82, 0x24, 0x5b, 0x62, 0x64, + 0x02, 0x5e, 0x99, 0x12, 0x37, 0x16, 0xed, 0xce, 0xc9, 0x88, 0x4a, 0x61, 0x2e, 0xf2, 0xca, 0xa7, + 0xd6, 0x79, 0x75, 0x91, 0xf9, 0x81, 0xeb, 0x02, 0x5e, 0xe5, 0x78, 0xf8, 0x00, 0x96, 0x86, 0x86, + 0x69, 0xfb, 0x86, 0x69, 0x13, 0xc7, 0xed, 0x53, 0xd7, 0xb4, 0x07, 0x7c, 0x00, 0x85, 0xd3, 0x21, + 0x20, 0x6b, 0x92, 0x3a, 0x65, 0xa2, 0xe4, 0x9f, 0x3b, 0x51, 0xaa, 0xf0, 0xb6, 0x45, 0xf7, 0x7d, + 0xc2, 0x37, 0xd7, 0x23, 0xd3, 0x3f, 0x20, 0x23, 0xc3, 0x74, 0x69, 0x9f, 0x23, 0xa8, 0x3b, 0x31, + 0x8f, 0x8a, 0x8c, 0x93, 0x45, 0xef, 0xa1, 0xe9, 0x1f, 0xb4, 0x38, 0xdb, 0x16, 0xe7, 0xc2, 0x0f, + 0xe1, 0x86, 0x33, 0xf6, 0x47, 0x63, 0x9f, 0x0c, 0x5c, 0x87, 0xa7, 0x9f, 0xed, 0x9b, 0xf6, 0xd8, + 0xf0, 0x4d, 0xc7, 0x26, 0xfb, 0x8e, 0x4b, 0xf8, 0x19, 0xae, 0x73, 0x54, 0x5c, 0x8c, 0xe8, 0xbb, + 0x2a, 0x64, 0xea, 0x4c, 0xa4, 0x12, 0x91, 0xa8, 0x39, 0x6e, 0x93, 0xee, 0xfb, 0xba, 0x73, 0x84, + 0xff, 0x07, 0x8a, 0x32, 0xa3, 0xbf, 0xbb, 0x5b, 0xe0, 0xc8, 0x6e, 0x71, 0x49, 0x70, 0x6d, 0x4e, + 0x6e, 0x18, 0x5b, 0xc9, 0x74, 0x0a, 0xcd, 0x6f, 0x25, 0xd3, 0x39, 0x94, 0x67, 0x5b, 0x2d, 0xb4, + 0x1d, 0xd7, 0x97, 0x4d, 0xe6, 0x01, 0x2c, 0x4a, 0x93, 0x43, 0xef, 0x8a, 0x76, 0x53, 0x9a, 0x9e, + 0x75, 0x81, 0xa7, 0xe5, 0xa1, 0x05, 0xa1, 0x20, 0xea, 0xff, 0x40, 0x17, 0x19, 0x72, 0x73, 0x2d, + 0x6a, 0x4f, 0x34, 0x1f, 0x14, 0xd0, 0xb7, 0x19, 0xb9, 0x49, 0xed, 0xd2, 0x2f, 0xe2, 0x90, 0xab, + 0x9a, 0x9e, 0x6f, 0xda, 0x3d, 0x9f, 0xdb, 0x75, 0x13, 0x16, 0x39, 0x53, 0x64, 0xc6, 0xc7, 0xf8, + 0x8c, 0x2f, 0x48, 0x74, 0x10, 0xb9, 0x5b, 0x80, 0xfa, 0x52, 0x30, 0xe4, 0x8c, 0x73, 0xce, 0xc5, + 0x00, 0x1f, 0xb0, 0x6e, 0x00, 0xb6, 0xc7, 0x96, 0x25, 0xda, 0x41, 0x40, 0x9c, 0x58, 0x65, 0x11, + 0xa7, 0x97, 0x5d, 0x1a, 0xd8, 0x82, 0x6f, 0x40, 0x8e, 0xba, 0xae, 0xe3, 0x12, 0xc7, 0x26, 0xfd, + 0xf1, 0x88, 0x77, 0x85, 0x4c, 0x50, 0x68, 0x9c, 0xa2, 0xd9, 0xd5, 0xf1, 0xe8, 0x3c, 0x3f, 0xa6, + 0x5e, 0xce, 0x8f, 0x25, 0x04, 0x05, 0xcd, 0xed, 0x9b, 0xb6, 0xc1, 0x2a, 0x99, 0x39, 0xa5, 0xf4, + 0xab, 0x04, 0xa0, 0x4f, 0xcc, 0xc1, 0xe7, 0xc6, 0x40, 0x24, 0x1c, 0xf7, 0x54, 0x15, 0xe6, 0x79, + 0xaf, 0x0f, 0x9e, 0x24, 0xb3, 0xcd, 0x09, 0x29, 0x8b, 0x6b, 0x00, 0xf4, 0x70, 0xc2, 0x81, 0xd9, + 0x8d, 0xab, 0xd3, 0x4d, 0x97, 0x2e, 0x0d, 0xf6, 0x72, 0x7a, 0x78, 0x1a, 0x8e, 0x82, 0x18, 0x38, + 0x8e, 0x30, 0x7d, 0xa2, 0x4d, 0x73, 0x8a, 0xbc, 0x93, 0xf7, 0x6a, 0x7a, 0xed, 0x7d, 0xc8, 0xed, + 0x9b, 0xc7, 0xb4, 0x4f, 0x9e, 0xf0, 0x97, 0x5a, 0x31, 0xc5, 0x2d, 0x7f, 0x4e, 0xcb, 0x9c, 0x7c, + 0xd1, 0xe9, 0x59, 0x2e, 0x2d, 0x90, 0x2f, 0xd1, 0xb8, 0x4b, 0x7f, 0x4c, 0xc0, 0xe2, 0x36, 0x75, + 0x07, 0x34, 0x12, 0x99, 0x6d, 0xc8, 0xf3, 0x82, 0x7f, 0xe1, 0xca, 0xca, 0x31, 0xf1, 0xb0, 0xae, + 0x34, 0x28, 0xb8, 0xe6, 0xe0, 0x20, 0xa2, 0x2f, 0x3e, 0xa3, 0xbe, 0x3c, 0x97, 0x0f, 0x15, 0x46, + 0x02, 0x90, 0x7a, 0x1d, 0xc3, 0xee, 0x16, 0xe4, 0x59, 0xbd, 0x11, 0x7a, 0x38, 0x36, 0xc2, 0x79, + 0x17, 0x94, 0x62, 0x8e, 0x91, 0x14, 0x49, 0xc1, 0x1f, 0xc3, 0x65, 0xee, 0xca, 0xd3, 0x1c, 0x9d, + 0x32, 0xd3, 0xe9, 0xbe, 0xaf, 0x1c, 0x4e, 0xce, 0xf4, 0xff, 0x86, 0xa2, 0xf0, 0xdb, 0x39, 0xc2, + 0x99, 0x88, 0xf0, 0x45, 0xce, 0x75, 0x46, 0xba, 0xf4, 0xb7, 0x38, 0x14, 0xee, 0x19, 0xde, 0x41, + 0x24, 0xae, 0xb7, 0x61, 0xf1, 0x8c, 0x31, 0xa2, 0x37, 0xc9, 0x5d, 0x24, 0x6a, 0x02, 0xbe, 0x03, + 0xe8, 0xec, 0xe1, 0xa2, 0x3d, 0x71, 0xe6, 0xc2, 0xe4, 0x91, 0xaf, 0x3d, 0x22, 0xaf, 0xcd, 0xcd, + 0x5b, 0xc9, 0xf4, 0x02, 0x4a, 0x97, 0xbe, 0x48, 0x02, 0x6e, 0xc8, 0x6f, 0x35, 0x11, 0x87, 0xff, + 0x48, 0x9b, 0xb0, 0x06, 0xf9, 0xe0, 0x43, 0xd1, 0x8b, 0xb6, 0xa5, 0x5c, 0xa0, 0x80, 0x47, 0xe2, + 0x75, 0x87, 0xf3, 0xdc, 0xe5, 0x69, 0xe1, 0xb9, 0xcb, 0xd3, 0xf7, 0x5f, 0x61, 0xd2, 0xb3, 0xad, + 0x30, 0x1f, 0xc3, 0xe5, 0x91, 0x4b, 0xf7, 0xcd, 0xe3, 0xb0, 0xdc, 0xc3, 0x7a, 0xc8, 0x84, 0xf5, + 0x70, 0x49, 0xb0, 0x04, 0x65, 0x2f, 0x53, 0x64, 0x2b, 0x99, 0x4e, 0xa0, 0x64, 0xe9, 0x8b, 0x04, + 0x5c, 0x0c, 0x92, 0x63, 0xe2, 0xed, 0xbd, 0x0e, 0x28, 0x8c, 0x5b, 0xcf, 0xb1, 0x78, 0x84, 0x63, + 0x91, 0x08, 0x17, 0x02, 0x6a, 0xc5, 0xb1, 0x58, 0x9c, 0x77, 0xce, 0xc6, 0x59, 0xf4, 0xd1, 0x77, + 0xcf, 0xb8, 0x36, 0xe0, 0x59, 0x67, 0xcf, 0xdb, 0xd3, 0xf0, 0xb4, 0x5c, 0xc7, 0x77, 0xce, 0x0d, + 0xf7, 0x67, 0xb0, 0x34, 0x72, 0x29, 0xd9, 0x97, 0xb6, 0x11, 0x6f, 0x44, 0x7b, 0x3c, 0x6c, 0xd9, + 0x8d, 0xff, 0x9d, 0x1e, 0xf8, 0xf3, 0xae, 0xb4, 0xde, 0x72, 0x69, 0x14, 0xd6, 0x17, 0x47, 0x93, + 0x88, 0xe5, 0x9f, 0xc5, 0x60, 0xf1, 0x0c, 0x13, 0xde, 0x02, 0x38, 0xfd, 0xb4, 0xf9, 0x02, 0xdf, + 0x21, 0x22, 0xd2, 0x78, 0x5d, 0x66, 0x9d, 0x70, 0xcd, 0xf2, 0xd9, 0xac, 0xa3, 0xc3, 0x75, 0xf1, + 0x9d, 0xb6, 0x23, 0xe7, 0xdf, 0x1f, 0x00, 0x0a, 0xe5, 0xc1, 0xc0, 0xa5, 0x03, 0xc3, 0x77, 0x84, + 0x39, 0x57, 0x01, 0x82, 0x1c, 0xb2, 0xa2, 0x4d, 0x2f, 0x33, 0x10, 0x69, 0x62, 0x79, 0xf8, 0xff, + 0x21, 0x67, 0x48, 0x21, 0xd3, 0x09, 0x3f, 0x3a, 0xfc, 0xc7, 0x74, 0x9b, 0x27, 0x8f, 0x08, 0xc1, + 0x48, 0x01, 0x46, 0xf5, 0xe1, 0xf7, 0xe5, 0x2a, 0x4a, 0xfb, 0x24, 0x62, 0x4a, 0x32, 0x34, 0x05, + 0x49, 0x6a, 0x3d, 0xb4, 0xa8, 0x2e, 0xef, 0x9d, 0xe2, 0xd5, 0xf6, 0xde, 0xf7, 0xb6, 0xe4, 0x3b, + 0xb5, 0x77, 0xce, 0x42, 0x38, 0xff, 0x72, 0x0b, 0xe1, 0xf2, 0x4f, 0xe3, 0x90, 0x8d, 0xdc, 0x98, + 0xd9, 0xba, 0x3f, 0xb6, 0x7b, 0x3c, 0xd2, 0xb3, 0xd8, 0x5a, 0x1b, 0xdb, 0xbd, 0xc0, 0x56, 0xa6, + 0x00, 0xaf, 0x42, 0x3a, 0x5c, 0x87, 0xe3, 0x91, 0xb2, 0x0e, 0xb1, 0xf8, 0x32, 0xff, 0xf0, 0xcd, + 0x2b, 0x2b, 0xc5, 0x97, 0xeb, 0xf9, 0x9e, 0xa8, 0xa5, 0xeb, 0x50, 0x10, 0xf9, 0x1e, 0x56, 0x1e, + 0x6b, 0x9a, 0x79, 0x3d, 0x27, 0xb0, 0xb2, 0xe2, 0xee, 0x41, 0xc6, 0x70, 0x07, 0xe3, 0x21, 0xb5, + 0x7d, 0xaf, 0x38, 0xcf, 0x83, 0x3c, 0x4b, 0x62, 0x9e, 0x0a, 0xcb, 0x56, 0xf0, 0xdb, 0x14, 0x24, + 0xd9, 0x2d, 0x30, 0x82, 0x5c, 0x59, 0x7d, 0x44, 0x54, 0xad, 0x43, 0xd4, 0x6e, 0xb3, 0x89, 0xe6, + 0xf0, 0x02, 0x24, 0xca, 0x3b, 0x75, 0x14, 0xc3, 0x39, 0x48, 0x6f, 0x6a, 0x5a, 0x93, 0x94, 0xd5, + 0x2a, 0x8a, 0xe3, 0x2c, 0x2c, 0x70, 0x48, 0xd3, 0x51, 0x02, 0x17, 0x00, 0x2a, 0x9a, 0x5a, 0x29, + 0x77, 0x48, 0xb9, 0x5e, 0x47, 0x49, 0x9c, 0x81, 0x54, 0x45, 0xeb, 0xaa, 0x1d, 0x94, 0x62, 0xe2, + 0xdb, 0xe5, 0x5d, 0xb4, 0xc0, 0xff, 0x68, 0xa8, 0x28, 0x8d, 0x01, 0xe6, 0xdb, 0x9d, 0x6a, 0x55, + 0xd9, 0x41, 0x19, 0x86, 0x6c, 0x77, 0xb7, 0x11, 0x30, 0x75, 0xed, 0xee, 0x36, 0x69, 0xa8, 0x1d, + 0x94, 0x65, 0x27, 0xed, 0x94, 0xf5, 0x46, 0x59, 0xad, 0x28, 0x28, 0xc7, 0x48, 0xbb, 0x9a, 0xce, + 0x35, 0xe7, 0xc5, 0x49, 0x5d, 0xb5, 0x43, 0x74, 0xed, 0x61, 0x1b, 0x15, 0xb8, 0xdc, 0x03, 0xbd, + 0xda, 0xa8, 0xd5, 0xd0, 0x22, 0xc6, 0x50, 0xa8, 0x35, 0xd4, 0x72, 0x93, 0x84, 0xd2, 0x88, 0x5d, + 0x48, 0xe0, 0xe4, 0x99, 0x4b, 0x38, 0x0f, 0x99, 0xb2, 0xae, 0x97, 0x1f, 0x71, 0x8d, 0x98, 0x1d, + 0xb6, 0xd5, 0xd6, 0x54, 0x0e, 0x5d, 0x60, 0x44, 0x06, 0x6d, 0x72, 0xf0, 0x22, 0x3b, 0xae, 0xdd, + 0xd1, 0x1b, 0x6a, 0x9d, 0xc3, 0x97, 0xf8, 0xad, 0x1b, 0x1d, 0xee, 0x82, 0x37, 0xd8, 0x45, 0x18, + 0xa0, 0xe9, 0xe8, 0x32, 0x4e, 0x43, 0xb2, 0xa2, 0xe9, 0x3a, 0x2a, 0xe2, 0x22, 0x5c, 0x6c, 0x29, + 0x7a, 0x45, 0x51, 0x3b, 0x8d, 0xa6, 0x42, 0xaa, 0x8d, 0x76, 0x85, 0x34, 0xb6, 0x5b, 0x4d, 0xf4, + 0xe6, 0x19, 0x4a, 0x45, 0x53, 0x3b, 0x82, 0xb2, 0x8c, 0x2f, 0xc0, 0x22, 0xb7, 0x41, 0xdb, 0xdc, + 0x52, 0x2a, 0xc2, 0x89, 0x6f, 0xe1, 0x8b, 0x80, 0x84, 0x29, 0x11, 0xec, 0xdb, 0xcc, 0x82, 0x9d, + 0xb2, 0x4e, 0x5a, 0x5a, 0x0b, 0xfd, 0x9b, 0x30, 0x8f, 0x5d, 0x8b, 0xc3, 0x2b, 0x78, 0x11, 0xb2, + 0xed, 0x0e, 0xd9, 0x2e, 0xdf, 0x57, 0x9a, 0x0d, 0x55, 0x41, 0x57, 0xd8, 0x75, 0xda, 0x1d, 0xa2, + 0xec, 0x76, 0x14, 0xb5, 0x83, 0x56, 0xd9, 0x5d, 0xdb, 0x1d, 0xd2, 0x55, 0x1b, 0x9a, 0x8a, 0xae, + 0x0a, 0x69, 0x52, 0xd1, 0x9a, 0x4d, 0xa5, 0xd2, 0x41, 0x25, 0xc6, 0x5c, 0xd1, 0x02, 0xe5, 0xd7, + 0x84, 0xab, 0x19, 0xd8, 0x2e, 0x6f, 0xb7, 0xd0, 0x75, 0xe6, 0x5d, 0x5d, 0xa9, 0xeb, 0x2c, 0x46, + 0xec, 0x16, 0xad, 0x0e, 0x7a, 0x87, 0x59, 0xc3, 0x71, 0xfa, 0x06, 0xba, 0xc1, 0x04, 0x38, 0xd0, + 0x6e, 0x6a, 0x2d, 0x05, 0xdd, 0x64, 0xa7, 0x09, 0x78, 0x77, 0x17, 0xad, 0x9d, 0x42, 0x8f, 0x1e, + 0xa1, 0x5b, 0x11, 0xda, 0x23, 0x74, 0x3b, 0x94, 0x14, 0x49, 0xf3, 0x2e, 0xb3, 0x84, 0xc3, 0xe5, + 0x9d, 0xfa, 0x2e, 0xba, 0x13, 0x05, 0x1f, 0xa1, 0xf7, 0x4a, 0x77, 0x20, 0xc9, 0xba, 0x03, 0xf3, + 0x79, 0xb9, 0xdb, 0xd1, 0xd0, 0x1c, 0x4f, 0xa9, 0x4a, 0xb9, 0x59, 0xd6, 0x51, 0x8c, 0xe9, 0x52, + 0x35, 0x95, 0x48, 0x38, 0x5e, 0xfa, 0x7d, 0x0c, 0x0a, 0x2d, 0xd7, 0xf9, 0x8c, 0xf6, 0xfc, 0x36, + 0x15, 0x6f, 0xe1, 0xff, 0x83, 0x14, 0xeb, 0xcc, 0xc1, 0x03, 0x6f, 0x96, 0xca, 0x11, 0x82, 0xb8, + 0x0e, 0x4b, 0x03, 0x6a, 0x53, 0xd7, 0xf0, 0x23, 0xef, 0x69, 0xf1, 0xc8, 0x7b, 0x5e, 0x6b, 0x47, + 0xa1, 0x50, 0xb0, 0xa0, 0xde, 0x04, 0x64, 0x8f, 0xf9, 0x47, 0x15, 0x8f, 0x8c, 0xa8, 0x4b, 0x06, + 0xd4, 0x16, 0x0f, 0x3c, 0x3d, 0x6f, 0x8f, 0x87, 0xac, 0x83, 0xb6, 0xa8, 0x5b, 0xa7, 0x76, 0xe9, + 0x9b, 0x3c, 0xe4, 0x1e, 0x9a, 0x76, 0xdf, 0x39, 0x92, 0xc3, 0x69, 0x95, 0xff, 0x7e, 0xe2, 0x9b, + 0xbc, 0x57, 0x9f, 0xc8, 0xc7, 0x7c, 0x14, 0x85, 0xdb, 0x90, 0x39, 0xe2, 0x12, 0xb5, 0xd0, 0xb8, + 0xbb, 0xd3, 0xaf, 0x1a, 0x55, 0x2e, 0x81, 0x5a, 0xd8, 0x2f, 0x42, 0x3d, 0xcb, 0xbf, 0x8b, 0xc9, + 0x4e, 0xd1, 0x86, 0x7c, 0x30, 0x1a, 0x68, 0xed, 0x45, 0xbb, 0xa6, 0x3e, 0xa9, 0x03, 0x3f, 0x00, + 0x90, 0x47, 0x31, 0x8d, 0x71, 0xae, 0xf1, 0x83, 0xd9, 0x6c, 0x66, 0x5a, 0x23, 0x4a, 0x3e, 0x4e, + 0x3e, 0xfd, 0xf2, 0x4a, 0x6c, 0xf9, 0xcb, 0x05, 0x48, 0xd5, 0x5c, 0x63, 0x48, 0xf1, 0x7d, 0x48, + 0x0e, 0x9d, 0x3e, 0x95, 0xe6, 0x7e, 0x5f, 0xe5, 0x5c, 0x76, 0x7d, 0xdb, 0xe9, 0x87, 0x43, 0x89, + 0x29, 0xc1, 0x0f, 0x60, 0x7e, 0xcf, 0x19, 0xdb, 0x7d, 0x4f, 0xce, 0xf5, 0x7f, 0x9f, 0x49, 0xdd, + 0x26, 0x17, 0x0d, 0xd6, 0x53, 0xa1, 0x08, 0x7f, 0x02, 0x19, 0x7a, 0xdc, 0xb3, 0xc6, 0x7c, 0xe7, + 0x48, 0x70, 0x23, 0x3f, 0x9c, 0x49, 0xab, 0x12, 0x48, 0x87, 0x1f, 0x13, 0x02, 0xc4, 0xf2, 0xdf, + 0x63, 0x90, 0xe2, 0x87, 0xb2, 0x53, 0xf8, 0x79, 0xac, 0x90, 0xa4, 0x2b, 0x3e, 0x9c, 0xdd, 0xf6, + 0xc8, 0x90, 0x3e, 0x55, 0x87, 0xaf, 0x01, 0x98, 0xb6, 0x4f, 0x9c, 0xfd, 0x7d, 0x8f, 0x8a, 0xf9, + 0x17, 0xfc, 0xe4, 0x96, 0x31, 0x6d, 0x5f, 0xe3, 0x68, 0x7c, 0x15, 0x72, 0xac, 0x2a, 0xfa, 0x01, + 0x1b, 0xbb, 0x69, 0x4e, 0xcf, 0x72, 0x9c, 0x64, 0xd9, 0x82, 0xac, 0x20, 0xf2, 0x9f, 0xbe, 0xe5, + 0xe3, 0x61, 0x86, 0x1f, 0x88, 0x41, 0x48, 0x33, 0x9b, 0x96, 0x7f, 0x1d, 0x83, 0x79, 0xe1, 0x6e, + 0xac, 0x42, 0xca, 0xf3, 0x0d, 0xd7, 0x97, 0x0b, 0xdd, 0xc6, 0xec, 0xd7, 0x0e, 0x7f, 0x9a, 0x61, + 0x6a, 0x70, 0x15, 0x12, 0xd4, 0xee, 0xcb, 0x04, 0x78, 0x01, 0x6d, 0x3a, 0x13, 0x2f, 0xdd, 0x84, + 0x24, 0xcb, 0x2e, 0x36, 0x3a, 0xf5, 0xb2, 0x5a, 0x57, 0xd0, 0x1c, 0xeb, 0x6f, 0x7c, 0xca, 0xc5, + 0x58, 0x7f, 0xab, 0xeb, 0x5a, 0xb7, 0xd5, 0x46, 0xf1, 0xd2, 0xe7, 0x90, 0x09, 0x7d, 0x8f, 0x2f, + 0xc3, 0x85, 0xae, 0xba, 0xa9, 0x75, 0xd5, 0xaa, 0x52, 0x25, 0x2d, 0x5d, 0xa9, 0x28, 0xd5, 0x86, + 0x5a, 0x47, 0x73, 0x93, 0x84, 0x9a, 0xd6, 0x6c, 0x6a, 0x0f, 0x19, 0x21, 0xc6, 0xa6, 0x8a, 0x56, + 0xab, 0xb5, 0x95, 0x4e, 0x84, 0x3d, 0x1e, 0xc1, 0x9e, 0xf2, 0x26, 0xd8, 0x38, 0xa9, 0x74, 0x75, + 0x5d, 0x11, 0xe3, 0x16, 0x25, 0x4b, 0x9f, 0x42, 0x26, 0xcc, 0x2e, 0x36, 0x59, 0x55, 0x8d, 0x28, + 0xbb, 0x95, 0x66, 0xb7, 0xcd, 0x06, 0x0a, 0x3f, 0x94, 0x83, 0x55, 0x85, 0x44, 0xe5, 0x62, 0x78, + 0x09, 0xf2, 0x01, 0x81, 0xdf, 0x03, 0xc5, 0x99, 0x74, 0x80, 0xea, 0x34, 0x94, 0x36, 0x4a, 0x2c, + 0xff, 0x29, 0x0e, 0xe9, 0xa0, 0xef, 0x60, 0x25, 0xb2, 0x8a, 0x7d, 0xf7, 0x25, 0x31, 0xd5, 0xab, + 0x67, 0x17, 0xb1, 0x65, 0x48, 0x1b, 0xee, 0xc0, 0x6b, 0xf4, 0x8f, 0xbd, 0xe2, 0x02, 0xef, 0x90, + 0x21, 0x8c, 0xab, 0x90, 0x0e, 0x37, 0xc9, 0xe4, 0x8c, 0x9b, 0x64, 0x28, 0xc9, 0x9e, 0xd2, 0xfb, + 0x2c, 0x96, 0xf2, 0x41, 0x7a, 0x67, 0x96, 0xf8, 0xeb, 0x42, 0x14, 0xaf, 0xc1, 0xc4, 0x76, 0xc7, + 0xf7, 0xda, 0x54, 0xb0, 0x7f, 0x4f, 0xec, 0x7d, 0x6b, 0x90, 0x13, 0x3b, 0xac, 0xe4, 0x4c, 0x47, + 0x5e, 0x65, 0x13, 0x94, 0xad, 0x64, 0x3a, 0x8e, 0x12, 0x72, 0xbb, 0xfb, 0x4d, 0x0c, 0xe0, 0xb4, + 0x3b, 0xf2, 0x69, 0xab, 0x3d, 0x24, 0x6a, 0x77, 0x7b, 0x53, 0xd1, 0x65, 0x9e, 0x95, 0xd5, 0xfb, + 0x62, 0x76, 0x56, 0x15, 0xb5, 0xad, 0x10, 0x0e, 0xf3, 0x20, 0xc9, 0x8d, 0x45, 0x60, 0x12, 0x7c, + 0x47, 0xe8, 0x6e, 0xf3, 0xbd, 0xa6, 0x23, 0x16, 0x3d, 0xbe, 0xcd, 0x88, 0x45, 0xaf, 0x59, 0xae, + 0xa3, 0x79, 0xa6, 0xae, 0xa9, 0x94, 0xab, 0x68, 0x81, 0xe5, 0x4f, 0xad, 0xa1, 0xb7, 0x3b, 0x64, + 0xa7, 0xdc, 0xec, 0x2a, 0x28, 0xcd, 0xf4, 0x37, 0xcb, 0x21, 0x9c, 0x61, 0xda, 0xd4, 0xce, 0x3d, + 0x09, 0xc2, 0xed, 0xff, 0x84, 0xc2, 0xe4, 0x0f, 0x30, 0x2c, 0xf1, 0x5b, 0xdd, 0xcd, 0x66, 0xa3, + 0x82, 0xe6, 0xf0, 0x9b, 0x70, 0x49, 0xfc, 0xcd, 0xd6, 0x2f, 0xbe, 0xa1, 0x4a, 0x52, 0x6c, 0xf3, + 0xbd, 0xa7, 0xdf, 0xac, 0xcc, 0x3d, 0x7d, 0xb6, 0x12, 0xfb, 0xea, 0xd9, 0x4a, 0xec, 0xeb, 0x67, + 0x2b, 0xb1, 0xbf, 0x3c, 0x5b, 0x89, 0xfd, 0xfc, 0xdb, 0x95, 0xb9, 0xaf, 0xbe, 0x5d, 0x99, 0xfb, + 0xfa, 0xdb, 0x95, 0xb9, 0x4f, 0xb2, 0x91, 0xff, 0xd8, 0xf2, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, + 0x08, 0x1f, 0x69, 0xa3, 0xc8, 0x23, 0x00, 0x00, } func (m *ValuesCoreSpec) Marshal() (dAtA []byte, err error) { @@ -2250,6 +2263,11 @@ func (m *TableReaderSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.BatchBytesLimit)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x88 if m.VirtualColumn != nil { { size, err := m.VirtualColumn.MarshalToSizedBuffer(dAtA[:i]) @@ -2468,6 +2486,11 @@ func (m *JoinReaderSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + i = encodeVarintProcessorsSql(dAtA, i, uint64(m.LookupBatchBytesLimit)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x90 { size, err := m.RemoteLookupExpr.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -3702,6 +3725,7 @@ func (m *TableReaderSpec) Size() (n int) { l = m.VirtualColumn.Size() n += 2 + l + sovProcessorsSql(uint64(l)) } + n += 2 + sovProcessorsSql(uint64(m.BatchBytesLimit)) return n } @@ -3769,6 +3793,7 @@ func (m *JoinReaderSpec) Size() (n int) { n += 2 + l + sovProcessorsSql(uint64(l)) l = m.RemoteLookupExpr.Size() n += 2 + l + sovProcessorsSql(uint64(l)) + n += 2 + sovProcessorsSql(uint64(m.LookupBatchBytesLimit)) return n } @@ -4709,6 +4734,25 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 17: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchBytesLimit", wireType) + } + m.BatchBytesLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BatchBytesLimit |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -5458,6 +5502,25 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 18: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LookupBatchBytesLimit", wireType) + } + m.LookupBatchBytesLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LookupBatchBytesLimit |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index e348c922bd31..31039ca2cf8c 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -98,6 +98,11 @@ message TableReaderSpec { // limit hint. optional bool parallelize = 12 [(gogoproto.nullable) = false]; + // batch_bytes_limit, if non-zero, controls the TargetBytes limits that the + // TableReader will use for its scans. If zero, then the server-side default + // is used. If parallelize is set, this cannot be set. + optional int64 batch_bytes_limit = 17 [(gogoproto.nullable) = false]; + // If non-zero, this enables inconsistent historical scanning where different // batches can be read with different timestamps. This is used for // long-running table statistics which may outlive the TTL. Using this setting @@ -403,6 +408,13 @@ message JoinReaderSpec { // OutputGroupContinuationForLeftRow is true, MaintainOrdering must also // be true. optional bool output_group_continuation_for_left_row = 15 [(gogoproto.nullable) = false]; + + // lookup_batch_bytes_limit, if non-zero, controls the TargetBytes limits that + // the joiner will use for its lookups. If zero, then the server-side default + // is used. Note that, regardless of this setting, bytes limits are not always + // used for lookups - it depends on whether the joiner decides it wants + // DistSender-parallelism or not. + optional int64 lookup_batch_bytes_limit = 18 [(gogoproto.nullable) = false]; } // SorterSpec is the specification for a "sorting aggregator". A sorting diff --git a/pkg/sql/indexbackfiller_test.go b/pkg/sql/indexbackfiller_test.go index a7d73632b05e..c56e2000a1b5 100644 --- a/pkg/sql/indexbackfiller_test.go +++ b/pkg/sql/indexbackfiller_test.go @@ -400,7 +400,7 @@ INSERT INTO foo VALUES (1), (10), (100); )) require.NoError(t, fetcher.StartScan( - ctx, txn, spans, false, 0, true, false, /* forceProductionBatchSize */ + ctx, txn, spans, row.NoBytesLimit, 0, true, false, /* forceProductionBatchSize */ )) var rows []tree.Datums for { diff --git a/pkg/sql/logictest/BUILD.bazel b/pkg/sql/logictest/BUILD.bazel index a383fae805f8..c6aebbd52bfc 100644 --- a/pkg/sql/logictest/BUILD.bazel +++ b/pkg/sql/logictest/BUILD.bazel @@ -27,7 +27,6 @@ go_library( "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/randgen", - "//pkg/sql/row", "//pkg/sql/sem/tree", "//pkg/sql/sessiondatapb", "//pkg/sql/stats", diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 70df64dc22a7..8cfe701da158 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -53,7 +53,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/randgen" - "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/stats" @@ -266,10 +265,6 @@ import ( // - traceoff // Stops tracing. // -// - kv-batch-size -// Limits the kvfetcher batch size; it can be used to trigger certain error -// conditions or corner cases around limited batches. -// // - subtest // Defines the start of a subtest. The subtest is any number of statements // that occur after this command until the end of file or the next subtest @@ -2379,22 +2374,6 @@ func (t *logicTest) processSubtest( } t.traceStop() - case "kv-batch-size": - // kv-batch-size limits the kvfetcher batch size. It can be used to - // trigger certain error conditions around limited batches. - if len(fields) != 2 { - return errors.Errorf( - "kv-batch-size needs an integer argument, found: %v", - fields[1:], - ) - } - batchSize, err := strconv.Atoi(fields[1]) - if err != nil { - return errors.Errorf("kv-batch-size needs an integer argument; %s", err) - } - t.outf("Setting kv batch size %d", batchSize) - defer row.TestingSetKVBatchSize(int64(batchSize))() - default: return errors.Errorf("%s:%d: unknown command: %s", path, s.line+subtest.lineLineIndexIntoFile, cmd, diff --git a/pkg/sql/logictest/testdata/logic_test/select_index_span_ranges b/pkg/sql/logictest/testdata/logic_test/select_index_span_ranges deleted file mode 100644 index 7309f73d162c..000000000000 --- a/pkg/sql/logictest/testdata/logic_test/select_index_span_ranges +++ /dev/null @@ -1,66 +0,0 @@ -# LogicTest: !3node-tenant - -# This test verifies that we correctly perform an index join when the KV -# batches span ranges. This is testing that SQL disables batch limits for index -# join; otherwise it can get out of order results from KV that it can't handle. - -kv-batch-size 10 - -statement ok -CREATE TABLE t ( - a INT PRIMARY KEY, - b INT, - c INT, - d INT, - FAMILY (a), - FAMILY (b), - FAMILY (c), - FAMILY (d), - INDEX c (c) -) - -statement ok -INSERT INTO t VALUES -(1, 0, 99, 0), -(2, 0, 80, 0), -(3, 0, 90, 0), -(4, 0, 10, 0), -(5, 0, 20, 0), -(6, 0, 85, 0), -(7, 0, 91, 0), -(8, 0, 12, 0), -(9, 0, 91, 0), -(10, 0, 11, 0), -(11, 0, 12, 0), -(12, 0, 88, 0), -(13, 0, 13, 0) - -# Split the table across multiple ranges. -statement ok -ALTER TABLE t SPLIT AT VALUES (2) - -statement ok -ALTER TABLE t SPLIT AT VALUES (3) - -statement ok -ALTER TABLE t SPLIT AT VALUES (5) - -statement ok -ALTER TABLE t SPLIT AT VALUES (8) - -statement ok -ALTER INDEX t@c SPLIT AT VALUES (90) - -statement ok -ALTER INDEX c SPLIT AT VALUES (10) - -query IIII partialsort(3) -SELECT * FROM t@c WHERE (c >= 80) ORDER BY c ----- -2 0 80 0 -6 0 85 0 -12 0 88 0 -3 0 90 0 -7 0 91 0 -9 0 91 0 -1 0 99 0 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_nonmetamorphic index aa89ee873a80..00e9325450b1 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join_nonmetamorphic @@ -1,61 +1,5 @@ # LogicTest: 5node !metamorphic -# Regression test for #35950: Make sure that lookup joins use a batch limit. - -statement ok -CREATE TABLE a (a INT, b INT, PRIMARY KEY (a, b)) - -statement ok -CREATE TABLE b (a INT PRIMARY KEY) - -# We insert over 10k rows, which is the currently configured batch limit. - -statement ok -INSERT INTO a SELECT 1, g FROM generate_series(1,11000) g - -statement ok -INSERT INTO b VALUES(1) - -query T -EXPLAIN SELECT count(*) FROM (SELECT * FROM b NATURAL INNER LOOKUP JOIN a) ----- -distribution: full -vectorized: true -· -• group (scalar) -│ -└── • lookup join - │ table: a@primary - │ equality: (a) = (a) - │ - └── • scan - missing stats - table: b@primary - spans: FULL SCAN - -statement ok -SET tracing = on - -query I -SELECT count(*) FROM (SELECT * FROM b NATURAL INNER LOOKUP JOIN a) ----- -11000 - -statement ok -SET tracing = off - -let $lookupTableID -SELECT 'a'::regclass::oid - -# Now assert that we get more than 1 separate batch request into the lookup -# table, since the first one wouldn't have returned all of the results. - -query T -SELECT message FROM [SHOW TRACE FOR SESSION] WHERE message LIKE 'Scan /Table/$lookupTableID%' ----- -Scan /Table/53/1/{1-2} -Scan /Table/53/1/{1/10001/0-2} - # The following tests check that if the joiners can separate a row request # into separate families that it does, and generates spans for each family # instead of reading the entire row when it doesn't need to. @@ -75,8 +19,8 @@ INSERT INTO family_split_2 VALUES (1, 2, 3) query T kvtrace(Scan) SELECT family_split_2.x, family_split_2.z FROM family_split_1 INNER LOOKUP JOIN family_split_2 ON family_split_1.x = family_split_2.x; SET tracing = off ---- -Scan /Table/55/{1-2} -Scan /Table/56/1/1/0, /Table/56/1/1/2/1 +Scan /Table/53/{1-2} +Scan /Table/54/1/1/0, /Table/54/1/1/2/1 statement ok CREATE TABLE family_index_join (x INT PRIMARY KEY, y INT, z INT, w INT, INDEX (y), FAMILY f1 (x), FAMILY f2 (y), FAMILY f3 (z), FAMILY f4(w)) @@ -87,8 +31,8 @@ INSERT INTO family_index_join VALUES (1, 2, 3, 4) query T kvtrace(Scan) SELECT y,w FROM family_index_join@family_index_join_y_idx WHERE y = 2 ---- -Scan /Table/57/2/{2-3} -Scan /Table/57/1/1/{0-1/2}, /Table/57/1/1/3/1 +Scan /Table/55/2/{2-3} +Scan /Table/55/1/1/{0-1/2}, /Table/55/1/1/3/1 # Test generating tighter spans on interleaved tables. statement ok @@ -106,5 +50,5 @@ INSERT INTO family_interleave_2 VALUES (1, 3) query T kvtrace(Scan) SELECT family_interleave_1.x, family_interleave_1.z FROM family_interleave_2 INNER LOOKUP JOIN family_interleave_1 ON family_interleave_1.x = family_interleave_2.x ---- -Scan /Table/58/{1-2} -Scan /Table/58/1/1/0, /Table/58/1/1/2/1 +Scan /Table/56/{1-2} +Scan /Table/56/1/1/0, /Table/56/1/1/2/1 diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index c82b4b0220ca..4ff8ea0d6411 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -563,14 +563,51 @@ func (rf *Fetcher) GetTables() []catalog.Descriptor { return ret } +// RowLimit represents a response limit expressed in terms of number of result +// rows. RowLimits get ultimately converted to KeyLimits and are translated into +// BatchRequest.MaxSpanRequestKeys. +type RowLimit uint64 + +// KeyLimit represents a response limit expressed in terms of number of keys. +type KeyLimit int64 + +// BytesLimit represents a response limit expressed in terms of the size of the +// results. A BytesLimit ultimately translates into BatchRequest.TargetBytes. +type BytesLimit uint64 + +// NoRowLimit can be passed to Fetcher.StartScan to signify that the caller +// doesn't want to limit the number of result rows for each scan request. +const NoRowLimit RowLimit = 0 + +// NoBytesLimit can be passed to Fetcher.StartScan to signify that the caller +// doesn't want to limit the size of results for each scan request. +// +// See also DefaultBatchBytesLimit. +const NoBytesLimit BytesLimit = 0 + // StartScan initializes and starts the key-value scan. Can be used multiple // times. +// +// batchBytesLimit controls whether bytes limits are placed on the batches. If +// set, bytes limits will be used to protect against running out of memory (on +// both this client node, and on the server). +// +// If batchBytesLimit is set, rowLimitHint can also be set to control the number of +// rows that will be scanned by the first batch. If set, subsequent batches (if +// any) will have progressively higher limits (up to a fixed max). The idea with +// row limits is to make the execution of LIMIT queries efficient: if the caller +// has some idea about how many rows need to be read to ultimately satisfy the +// query, the Fetcher uses it. Even if this hint proves insufficient, the +// Fetcher continues to set row limits (in addition to bytes limits) on the +// argument that some number of rows will eventually satisfy the query and we +// likely don't need to scan `spans` fully. The bytes limit, on the other hand, +// is simply intended to protect against OOMs. func (rf *Fetcher) StartScan( ctx context.Context, txn *kv.Txn, spans roachpb.Spans, - limitBatches bool, - limitHint int64, + batchBytesLimit BytesLimit, + rowLimitHint RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error { @@ -583,8 +620,8 @@ func (rf *Fetcher) StartScan( makeKVBatchFetcherDefaultSendFunc(txn), spans, rf.reverse, - limitBatches, - rf.rowLimitToKeyLimit(limitHint), + batchBytesLimit, + rf.rowLimitToKeyLimit(rowLimitHint), rf.lockStrength, rf.lockWaitPolicy, rf.lockTimeout, @@ -621,8 +658,8 @@ func (rf *Fetcher) StartInconsistentScan( initialTimestamp hlc.Timestamp, maxTimestampAge time.Duration, spans roachpb.Spans, - limitBatches bool, - limitHint int64, + batchBytesLimit BytesLimit, + rowLimitHint RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error { @@ -683,8 +720,8 @@ func (rf *Fetcher) StartInconsistentScan( sendFunc(sendFn), spans, rf.reverse, - limitBatches, - rf.rowLimitToKeyLimit(limitHint), + batchBytesLimit, + rf.rowLimitToKeyLimit(rowLimitHint), rf.lockStrength, rf.lockWaitPolicy, rf.lockTimeout, @@ -699,19 +736,19 @@ func (rf *Fetcher) StartInconsistentScan( return rf.StartScanFrom(ctx, &f) } -func (rf *Fetcher) rowLimitToKeyLimit(keyLimitHint int64) int64 { - if keyLimitHint == 0 { +func (rf *Fetcher) rowLimitToKeyLimit(rowLimitHint RowLimit) KeyLimit { + if rowLimitHint == 0 { return 0 } // If we have a limit hint, we limit the first batch size. Subsequent // batches get larger to avoid making things too slow (e.g. in case we have // a very restrictive filter and actually have to retrieve a lot of rows). - // The keyLimitHint is a row limit, but each row could be made up of more than + // The rowLimitHint is a row limit, but each row could be made up of more than // one key. We take the maximum possible keys per row out of all the table // rows we could potentially scan over. // // We add an extra key to make sure we form the last row. - return keyLimitHint*int64(rf.maxKeysPerRow) + 1 + return KeyLimit(int64(rowLimitHint)*int64(rf.maxKeysPerRow) + 1) } // StartScanFrom initializes and starts a scan from the given kvBatchFetcher. Can be diff --git a/pkg/sql/row/fetcher_test.go b/pkg/sql/row/fetcher_test.go index 16b4f7245fcd..4068b4f89a5d 100644 --- a/pkg/sql/row/fetcher_test.go +++ b/pkg/sql/row/fetcher_test.go @@ -171,8 +171,8 @@ func TestNextRowSingle(t *testing.T) { context.Background(), kv.NewTxn(ctx, kvDB, 0), roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPrimaryIndexID())}, - false, /*limitBatches*/ - 0, /*limitHint*/ + NoBytesLimit, + NoRowLimit, false, /*traceKV*/ false, /*forceProductionKVBatchSize*/ ); err != nil { @@ -292,7 +292,7 @@ func TestNextRowBatchLimiting(t *testing.T) { context.Background(), kv.NewTxn(ctx, kvDB, 0), roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPrimaryIndexID())}, - true, /*limitBatches*/ + DefaultBatchBytesLimit, 10, /*limitHint*/ false, /*traceKV*/ false, /*forceProductionKVBatchSize*/ @@ -403,8 +403,8 @@ func TestRowFetcherMemoryLimits(t *testing.T) { context.Background(), kv.NewTxn(ctx, kvDB, 0), roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.GetPrimaryIndexID())}, - false, /*limitBatches*/ - 0, /*limitHint*/ + NoBytesLimit, + NoRowLimit, false, /*traceKV*/ false, /*forceProductionKVBatchSize*/ ) @@ -489,7 +489,7 @@ INDEX(c) roachpb.Spans{indexSpan, roachpb.Span{Key: midKey, EndKey: endKey}, }, - true, /*limitBatches*/ + DefaultBatchBytesLimit, // Set a limitHint of 1 to more quickly end the first batch, causing a // batch that ends between rows. 1, /*limitHint*/ @@ -652,8 +652,8 @@ func TestNextRowSecondaryIndex(t *testing.T) { context.Background(), kv.NewTxn(ctx, kvDB, 0), roachpb.Spans{tableDesc.IndexSpan(keys.SystemSQLCodec, tableDesc.PublicNonPrimaryIndexes()[0].GetID())}, - false, /*limitBatches*/ - 0, /*limitHint*/ + NoBytesLimit, + NoRowLimit, false, /*traceKV*/ false, /*forceProductionKVBatchSize*/ ); err != nil { @@ -1010,8 +1010,8 @@ func TestNextRowInterleaved(t *testing.T) { context.Background(), kv.NewTxn(ctx, kvDB, 0), lookupSpans, - false, /*limitBatches*/ - 0, /*limitHint*/ + NoBytesLimit, + NoRowLimit, false, /*traceKV*/ false, /*forceProductionKVBatchSize*/ ); err != nil { diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index 1a65b02a0f06..576cad778d25 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -36,28 +36,24 @@ import ( // only be used by tests the output of which differs if defaultKVBatchSize is // randomized. // TODO(radu): parameters like this should be configurable -func getKVBatchSize(forceProductionKVBatchSize bool) int64 { +func getKVBatchSize(forceProductionKVBatchSize bool) KeyLimit { if forceProductionKVBatchSize { return productionKVBatchSize } return defaultKVBatchSize } -var defaultKVBatchSize = int64(util.ConstantWithMetamorphicTestValue( +var defaultKVBatchSize = KeyLimit(util.ConstantWithMetamorphicTestValue( "kv-batch-size", - productionKVBatchSize, /* defaultValue */ - 1, /* metamorphicValue */ + int(productionKVBatchSize), /* defaultValue */ + 1, /* metamorphicValue */ )) -const productionKVBatchSize = 10000 +const productionKVBatchSize KeyLimit = 100000 -// TestingSetKVBatchSize changes the kvBatchFetcher batch size, and returns a function that restores it. -// This is to be used only in tests - we have no test coverage for arbitrary kv batch sizes at this time. -func TestingSetKVBatchSize(val int64) func() { - oldVal := defaultKVBatchSize - defaultKVBatchSize = val - return func() { defaultKVBatchSize = oldVal } -} +// DefaultBatchBytesLimit is the maximum number of bytes a scan request can +// return. +const DefaultBatchBytesLimit BytesLimit = 10 << 20 // 10 MB // sendFunc is the function used to execute a KV batch; normally // wraps (*client.Txn).Send. @@ -73,12 +69,23 @@ type txnKVFetcher struct { // individual Span has only a start key, it will be interpreted as a // single-key fetch and may use a GetRequest under the hood. spans roachpb.Spans - // If useBatchLimit is true, batches are limited to kvBatchSize. If - // firstBatchKeyLimit is also set, the first batch is limited to that value. - // Subsequent batches are larger, up to kvBatchSize. - firstBatchKeyLimit int64 - useBatchLimit bool - reverse bool + + // If firstBatchKeyLimit is set, the first batch is limited in number of keys + // to this value and subsequent batches are larger (up to a limit, see + // getKVBatchSize()). If not set, batches do not have a key limit (they might + // still have a bytes limit as per batchBytesLimit). + firstBatchKeyLimit KeyLimit + // If batchBytesLimit is set, the batches are limited in response size. This + // protects from OOMs, but comes at the cost of inhibiting DistSender-level + // parallelism within a batch. + // + // If batchBytesLimit is not set, the assumption is that SQL *knows* that + // there is only a "small" amount of data to be read (i.e. scanning `spans` + // doesn't result in too much data), and wants to preserve concurrency for + // this scans inside of DistSender. + batchBytesLimit BytesLimit + + reverse bool // lockStrength represents the locking mode to use when fetching KVs. lockStrength descpb.ScanLockingStrength // lockWaitPolicy represents the policy to be used for handling conflicting @@ -116,16 +123,17 @@ var _ kvBatchFetcher = &txnKVFetcher{} // getBatchKeyLimit returns the max size of the next batch. The size is // expressed in number of result keys (i.e. this size will be used for // MaxSpanRequestKeys). -func (f *txnKVFetcher) getBatchKeyLimit() int64 { +func (f *txnKVFetcher) getBatchKeyLimit() KeyLimit { return f.getBatchKeyLimitForIdx(f.batchIdx) } -func (f *txnKVFetcher) getBatchKeyLimitForIdx(batchIdx int) int64 { - if !f.useBatchLimit { +func (f *txnKVFetcher) getBatchKeyLimitForIdx(batchIdx int) KeyLimit { + if f.firstBatchKeyLimit == 0 { return 0 } + kvBatchSize := getKVBatchSize(f.forceProductionKVBatchSize) - if f.firstBatchKeyLimit == 0 || f.firstBatchKeyLimit >= kvBatchSize { + if f.firstBatchKeyLimit >= kvBatchSize { return kvBatchSize } @@ -232,8 +240,8 @@ func makeKVBatchFetcher( sendFn sendFunc, spans roachpb.Spans, reverse bool, - useBatchLimit bool, - firstBatchKeyLimit int64, + batchBytesLimit BytesLimit, + firstBatchKeyLimit KeyLimit, lockStrength descpb.ScanLockingStrength, lockWaitPolicy descpb.ScanLockingWaitPolicy, lockTimeout time.Duration, @@ -242,12 +250,15 @@ func makeKVBatchFetcher( requestAdmissionHeader roachpb.AdmissionHeader, responseAdmissionQ *admission.WorkQueue, ) (txnKVFetcher, error) { - if firstBatchKeyLimit < 0 || (!useBatchLimit && firstBatchKeyLimit != 0) { - return txnKVFetcher{}, errors.Errorf("invalid batch limit %d (useBatchLimit: %t)", - firstBatchKeyLimit, useBatchLimit) + if firstBatchKeyLimit < 0 || (batchBytesLimit == 0 && firstBatchKeyLimit != 0) { + // Passing firstBatchKeyLimit without batchBytesLimit doesn't make sense - the + // only reason to not set batchBytesLimit is in order to get DistSender-level + // parallelism, and setting firstBatchKeyLimit inhibits that. + return txnKVFetcher{}, errors.Errorf("invalid batch limit %d (batchBytesLimit: %d)", + firstBatchKeyLimit, batchBytesLimit) } - if useBatchLimit { + if batchBytesLimit != 0 { // Verify the spans are ordered if a batch limit is used. for i := 1; i < len(spans); i++ { prevKey := spans[i-1].EndKey @@ -302,7 +313,7 @@ func makeKVBatchFetcher( sendFn: sendFn, spans: copySpans, reverse: reverse, - useBatchLimit: useBatchLimit, + batchBytesLimit: batchBytesLimit, firstBatchKeyLimit: firstBatchKeyLimit, lockStrength: lockStrength, lockWaitPolicy: lockWaitPolicy, @@ -315,26 +326,13 @@ func makeKVBatchFetcher( }, nil } -// maxScanResponseBytes is the maximum number of bytes a scan request can -// return. -const maxScanResponseBytes = 10 * (1 << 20) // 10MB - // fetch retrieves spans from the kv layer. func (f *txnKVFetcher) fetch(ctx context.Context) error { var ba roachpb.BatchRequest ba.Header.WaitPolicy = f.getWaitPolicy() ba.Header.LockTimeout = f.lockTimeout - ba.Header.MaxSpanRequestKeys = f.getBatchKeyLimit() - if ba.Header.MaxSpanRequestKeys > 0 { - // If this kvfetcher limits the number of rows returned, also use - // target bytes to guard against the case in which the average row - // is very large. - // If no limit is set, the assumption is that SQL *knows* that there - // is only a "small" amount of data to be read, and wants to preserve - // concurrency for this request inside of DistSender, which setting - // TargetBytes would interfere with. - ba.Header.TargetBytes = maxScanResponseBytes - } + ba.Header.TargetBytes = int64(f.batchBytesLimit) + ba.Header.MaxSpanRequestKeys = int64(f.getBatchKeyLimit()) ba.AdmissionHeader = f.requestAdmissionHeader ba.Requests = make([]roachpb.RequestUnion, len(f.spans)) keyLocking := f.getKeyLockingStrength() @@ -453,9 +451,9 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { f.responses = nil } returnedBytes := int64(br.Size()) - if monitoring && (returnedBytes > maxScanResponseBytes || returnedBytes > f.acc.Used()) { + if monitoring && (returnedBytes > int64(f.batchBytesLimit) || returnedBytes > f.acc.Used()) { // Resize up to the actual amount of bytes we got back from the fetch, - // but don't ratchet down below maxScanResponseBytes if we ever exceed it. + // but don't ratchet down below f.batchBytesLimit if we ever exceed it. // We would much prefer to over-account than under-account, especially when // we are in a situation where we have large batches caused by parallel // unlimited scans (index joins and lookup joins where cols are key). diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 3fc133ad176f..b488cdce4e42 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -49,8 +49,8 @@ func NewKVFetcher( spans roachpb.Spans, bsHeader *roachpb.BoundedStalenessHeader, reverse bool, - useBatchLimit bool, - firstBatchLimit int64, + batchBytesLimit BytesLimit, + firstBatchLimit KeyLimit, lockStrength descpb.ScanLockingStrength, lockWaitPolicy descpb.ScanLockingWaitPolicy, lockTimeout time.Duration, @@ -77,7 +77,7 @@ func NewKVFetcher( sendFn, spans, reverse, - useBatchLimit, + batchBytesLimit, firstBatchLimit, lockStrength, lockWaitPolicy, diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index cfdd5927f004..b931d4cd3bfe 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -107,6 +107,7 @@ go_test( "inverted_expr_evaluator_test.go", "inverted_filterer_test.go", "inverted_joiner_test.go", + "joinreader_blackbox_test.go", "joinreader_test.go", "main_test.go", "mergejoiner_test.go", diff --git a/pkg/sql/rowexec/backfiller.go b/pkg/sql/rowexec/backfiller.go index dc60ac58a26c..5d9027eb3aab 100644 --- a/pkg/sql/rowexec/backfiller.go +++ b/pkg/sql/rowexec/backfiller.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -45,7 +46,7 @@ type chunkBackfiller interface { runChunk( ctx context.Context, span roachpb.Span, - chunkSize int64, + chunkSize row.RowLimit, readAsOf hlc.Timestamp, ) (roachpb.Key, error) @@ -147,7 +148,7 @@ func (b *backfiller) mainLoop(ctx context.Context) (roachpb.Spans, error) { for todo.Key != nil { log.VEventf(ctx, 3, "%s backfiller starting chunk %d: %s", b.name, chunks, todo) var err error - todo.Key, err = b.chunks.runChunk(ctx, todo, b.spec.ChunkSize, b.spec.ReadAsOf) + todo.Key, err = b.chunks.runChunk(ctx, todo, row.RowLimit(b.spec.ChunkSize), b.spec.ReadAsOf) if err != nil { return nil, err } diff --git a/pkg/sql/rowexec/columnbackfiller.go b/pkg/sql/rowexec/columnbackfiller.go index ee93d6aa2cac..7da002729454 100644 --- a/pkg/sql/rowexec/columnbackfiller.go +++ b/pkg/sql/rowexec/columnbackfiller.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -99,7 +100,7 @@ func (cb *columnBackfiller) CurrentBufferFill() float32 { // runChunk implements the chunkBackfiller interface. func (cb *columnBackfiller) runChunk( - ctx context.Context, sp roachpb.Span, chunkSize int64, _ hlc.Timestamp, + ctx context.Context, sp roachpb.Span, chunkSize row.RowLimit, _ hlc.Timestamp, ) (roachpb.Key, error) { var key roachpb.Key var commitWaitFn func(context.Context) error diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index 40f7fa23f395..abe35f1f7bbe 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -493,7 +493,7 @@ func (ij *invertedJoiner) readInput() (invertedJoinerState, *execinfrapb.Produce log.VEventf(ij.Ctx, 1, "scanning %d spans", len(indexSpans)) if err = ij.fetcher.StartScan( - ij.Ctx, ij.FlowCtx.Txn, indexSpans, false /* limitBatches */, 0, /* limitHint */ + ij.Ctx, ij.FlowCtx.Txn, indexSpans, row.NoBytesLimit, row.NoRowLimit, ij.FlowCtx.TraceKV, ij.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ); err != nil { ij.MoveToDraining(err) diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index e7265db5b8ff..78285820e4cb 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -173,6 +173,10 @@ type joinReader struct { // detailed comment in the spec). This can never be true for index joins, // and requires that the spec has MaintainOrdering set to true. outputGroupContinuationForLeftRow bool + // lookupBatchBytesLimit controls the TargetBytes of lookup requests. If 0, a + // default will be used. Regardless of this value, bytes limits aren't always + // used. + lookupBatchBytesLimit row.BytesLimit } var _ execinfra.Processor = &joinReader{} @@ -247,8 +251,9 @@ func newJoinReader( // row for each input row. Similarly, in case of spec.LookupColumnsAreKey, // we know that there's at most one lookup row per input row. In other // cases, we use limits. - shouldLimitBatches: !spec.LookupColumnsAreKey && readerType == lookupJoinReaderType, - readerType: readerType, + shouldLimitBatches: !spec.LookupColumnsAreKey && readerType == lookupJoinReaderType, + readerType: readerType, + lookupBatchBytesLimit: row.BytesLimit(spec.LookupBatchBytesLimit), } if readerType != indexJoinReaderType { jr.groupingState = &inputBatchGroupingState{doGrouping: spec.LeftJoinWithPairedJoiner} @@ -743,8 +748,17 @@ func (jr *joinReader) readInput() ( } log.VEventf(jr.Ctx, 1, "scanning %d spans", len(spans)) + var bytesLimit row.BytesLimit + if !jr.shouldLimitBatches { + bytesLimit = row.NoBytesLimit + } else { + bytesLimit = jr.lookupBatchBytesLimit + if jr.lookupBatchBytesLimit == 0 { + bytesLimit = row.DefaultBatchBytesLimit + } + } if err := jr.fetcher.StartScan( - jr.Ctx, jr.FlowCtx.Txn, spans, jr.shouldLimitBatches, 0, /* limitHint */ + jr.Ctx, jr.FlowCtx.Txn, spans, bytesLimit, row.NoRowLimit, jr.FlowCtx.TraceKV, jr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ); err != nil { jr.MoveToDraining(err) @@ -813,8 +827,12 @@ func (jr *joinReader) performLookup() (joinReaderState, *execinfrapb.ProducerMet sort.Sort(spans) log.VEventf(jr.Ctx, 1, "scanning %d remote spans", len(spans)) + bytesLimit := row.DefaultBatchBytesLimit + if !jr.shouldLimitBatches { + bytesLimit = row.NoBytesLimit + } if err := jr.fetcher.StartScan( - jr.Ctx, jr.FlowCtx.Txn, spans, jr.shouldLimitBatches, 0, /* limitHint */ + jr.Ctx, jr.FlowCtx.Txn, spans, bytesLimit, row.NoRowLimit, jr.FlowCtx.TraceKV, jr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ); err != nil { jr.MoveToDraining(err) diff --git a/pkg/sql/rowexec/joinreader_blackbox_test.go b/pkg/sql/rowexec/joinreader_blackbox_test.go new file mode 100644 index 000000000000..6664ee6fb334 --- /dev/null +++ b/pkg/sql/rowexec/joinreader_blackbox_test.go @@ -0,0 +1,99 @@ +// Copyright 2021 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 rowexec_test + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/stretchr/testify/require" +) + +// Check that the join reader uses bytes limits on its lookups. +func TestJoinReaderUsesBatchLimit(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + recCh := make(chan tracing.Recording, 1) + joinQuery := "SELECT count(1) FROM (SELECT * FROM test.b NATURAL INNER LOOKUP JOIN test.a)" + s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SQLExecutor: &sql.ExecutorTestingKnobs{ + // Get a recording for the join query. + WithStatementTrace: func(trace tracing.Recording, stmt string) { + if stmt == joinQuery { + recCh <- trace + } + }, + }, + // Make the join's bytes limit artificially low so that we don't need too + // big of a table to hit it. + DistSQL: &execinfra.TestingKnobs{ + JoinReaderBatchBytesLimit: 1000, + }, + }, + }) + defer s.Stopper().Stop(ctx) + + // We're going to create a table with enough rows to exceed a batch's memory + // limit. This table will represent the lookup side of a lookup join. + const numRows = 50 + sqlutils.CreateTable( + t, + sqlDB, + "a", + "a INT, b int, PRIMARY KEY (a,b)", + numRows, + // rows will look like (1, ) + sqlutils.ToRowFn( + func(row int) tree.Datum { + return tree.NewDInt(tree.DInt(1)) + }, + sqlutils.RowIdxFn, + ), + ) + sqlutils.CreateTable( + t, + sqlDB, + "b", + "a INT PRIMARY KEY", + 1, /* numRows */ + sqlutils.ToRowFn( + func(row int) tree.Datum { + return tree.NewDInt(tree.DInt(1)) + }), + ) + r := sqlDB.QueryRow(joinQuery) + var rows int + require.NoError(t, r.Scan(&rows)) + require.Equal(t, numRows, rows) + + // Look at the trace for the join and count how many (batch-)requests there + // were on the lookup side. We expect more than one of them (it would be only + // one if there was no limit on the size of results). + rec := <-recCh + desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "a") + tableID := desc.TableDesc().ID + sp, ok := rec.FindSpan("join reader") + require.True(t, ok) + require.Greater(t, tracing.CountLogMessages(sp, fmt.Sprintf("Scan /Table/%d", tableID)), 1) +} diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index 4d937173dbf2..89b29b24b766 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -32,8 +32,8 @@ import ( // collector wrapper can be plugged in. type rowFetcher interface { StartScan( - _ context.Context, _ *kv.Txn, _ roachpb.Spans, limitBatches bool, - limitHint int64, traceKV bool, forceProductionKVBatchSize bool, + _ context.Context, _ *kv.Txn, _ roachpb.Spans, batchBytesLimit row.BytesLimit, + rowLimitHint row.RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error StartInconsistentScan( _ context.Context, @@ -41,8 +41,8 @@ type rowFetcher interface { initialTimestamp hlc.Timestamp, maxTimestampAge time.Duration, spans roachpb.Spans, - limitBatches bool, - limitHint int64, + batchBytesLimit row.BytesLimit, + rowLimitHint row.RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index ba84c1731bc6..8a22dc172307 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -79,7 +79,7 @@ func newScrubTableReader( } tr.tableDesc = spec.BuildTableDescriptor() - tr.limitHint = execinfra.LimitHint(spec.LimitHint, post) + tr.limitHint = row.RowLimit(execinfra.LimitHint(spec.LimitHint, post)) if err := tr.Init( tr, @@ -218,7 +218,7 @@ func (tr *scrubTableReader) Start(ctx context.Context) { log.VEventf(ctx, 1, "starting") if err := tr.fetcher.StartScan( - ctx, tr.FlowCtx.Txn, tr.spans, true /* limit batches */, tr.limitHint, + ctx, tr.FlowCtx.Txn, tr.spans, row.DefaultBatchBytesLimit, tr.limitHint, tr.FlowCtx.TraceKV, tr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ); err != nil { tr.MoveToDraining(err) diff --git a/pkg/sql/rowexec/stats.go b/pkg/sql/rowexec/stats.go index e94223c01649..9d625e2c5868 100644 --- a/pkg/sql/rowexec/stats.go +++ b/pkg/sql/rowexec/stats.go @@ -108,13 +108,13 @@ func (c *rowFetcherStatCollector) StartScan( ctx context.Context, txn *kv.Txn, spans roachpb.Spans, - limitBatches bool, - limitHint int64, + batchBytesLimit row.BytesLimit, + limitHint row.RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error { start := timeutil.Now() - err := c.Fetcher.StartScan(ctx, txn, spans, limitBatches, limitHint, traceKV, forceProductionKVBatchSize) + err := c.Fetcher.StartScan(ctx, txn, spans, batchBytesLimit, limitHint, traceKV, forceProductionKVBatchSize) c.startScanStallTime += timeutil.Since(start) return err } @@ -126,14 +126,14 @@ func (c *rowFetcherStatCollector) StartInconsistentScan( initialTimestamp hlc.Timestamp, maxTimestampAge time.Duration, spans roachpb.Spans, - limitBatches bool, - limitHint int64, + batchBytesLimit row.BytesLimit, + limitHint row.RowLimit, traceKV bool, forceProductionKVBatchSize bool, ) error { start := timeutil.Now() err := c.Fetcher.StartInconsistentScan( - ctx, db, initialTimestamp, maxTimestampAge, spans, limitBatches, limitHint, traceKV, forceProductionKVBatchSize, + ctx, db, initialTimestamp, maxTimestampAge, spans, batchBytesLimit, limitHint, traceKV, forceProductionKVBatchSize, ) c.startScanStallTime += timeutil.Since(start) return err diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index c95f9a5d6478..e205a7d33dca 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -34,9 +34,10 @@ import ( type tableReader struct { execinfra.ProcessorBase - spans roachpb.Spans - limitHint int64 - parallelize bool + spans roachpb.Spans + limitHint row.RowLimit + parallelize bool + batchBytesLimit row.BytesLimit scanStarted bool @@ -80,12 +81,24 @@ func newTableReader( return nil, errors.Errorf("attempting to create a tableReader with uninitialized NodeID") } + if spec.LimitHint > 0 || spec.BatchBytesLimit > 0 { + // Parallelize shouldn't be set when there's a limit hint, but double-check + // just in case. + spec.Parallelize = false + } + var batchBytesLimit row.BytesLimit + if !spec.Parallelize { + batchBytesLimit = row.BytesLimit(spec.BatchBytesLimit) + if batchBytesLimit == 0 { + batchBytesLimit = row.DefaultBatchBytesLimit + } + } + tr := trPool.Get().(*tableReader) - tr.limitHint = execinfra.LimitHint(spec.LimitHint, post) - // Parallelize shouldn't be set when there's a limit hint, but double-check - // just in case. - tr.parallelize = spec.Parallelize && tr.limitHint == 0 + tr.limitHint = row.RowLimit(execinfra.LimitHint(spec.LimitHint, post)) + tr.parallelize = spec.Parallelize + tr.batchBytesLimit = batchBytesLimit tr.maxTimestampAge = time.Duration(spec.MaxTimestampAgeNanos) tableDesc := spec.BuildTableDescriptor() @@ -191,11 +204,17 @@ func (tr *tableReader) Start(ctx context.Context) { func (tr *tableReader) startScan(ctx context.Context) error { limitBatches := !tr.parallelize + var bytesLimit row.BytesLimit + if !limitBatches { + bytesLimit = row.NoBytesLimit + } else { + bytesLimit = tr.batchBytesLimit + } log.VEventf(ctx, 1, "starting scan with limitBatches %t", limitBatches) var err error if tr.maxTimestampAge == 0 { err = tr.fetcher.StartScan( - ctx, tr.FlowCtx.Txn, tr.spans, limitBatches, tr.limitHint, + ctx, tr.FlowCtx.Txn, tr.spans, bytesLimit, tr.limitHint, tr.FlowCtx.TraceKV, tr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ) @@ -203,7 +222,7 @@ func (tr *tableReader) startScan(ctx context.Context) error { initialTS := tr.FlowCtx.Txn.ReadTimestamp() err = tr.fetcher.StartInconsistentScan( ctx, tr.FlowCtx.Cfg.DB, initialTS, tr.maxTimestampAge, tr.spans, - limitBatches, tr.limitHint, tr.FlowCtx.TraceKV, + bytesLimit, tr.limitHint, tr.FlowCtx.TraceKV, tr.EvalCtx.TestingKnobs.ForceProductionBatchSizes, ) } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 1997666307c8..e6465f54cb92 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -252,7 +252,7 @@ type zigzagJoiner struct { // be fetched at a time. Increasing this will improve performance for when // matched rows are grouped together, but increasing this too much will result // in fetching too many rows and therefore skipping less rows. -var zigzagJoinerBatchSize = int64(util.ConstantWithMetamorphicTestValue( +var zigzagJoinerBatchSize = row.RowLimit(util.ConstantWithMetamorphicTestValue( "zig-zag-joiner-batch-size", 5, /* defaultValue */ 1, /* metamorphicValue */ @@ -793,7 +793,7 @@ func (z *zigzagJoiner) nextRow(ctx context.Context, txn *kv.Txn) (rowenc.EncDatu ctx, txn, roachpb.Spans{roachpb.Span{Key: curInfo.key, EndKey: curInfo.endKey}}, - true, /* batch limit */ + row.DefaultBatchBytesLimit, zigzagJoinerBatchSize, z.FlowCtx.TraceKV, z.EvalCtx.TestingKnobs.ForceProductionBatchSizes, @@ -938,7 +938,7 @@ func (z *zigzagJoiner) maybeFetchInitialRow() error { z.Ctx, z.FlowCtx.Txn, roachpb.Spans{roachpb.Span{Key: curInfo.key, EndKey: curInfo.endKey}}, - true, /* batch limit */ + row.DefaultBatchBytesLimit, zigzagJoinerBatchSize, z.FlowCtx.TraceKV, z.EvalCtx.TestingKnobs.ForceProductionBatchSizes, diff --git a/pkg/sql/scan_test.go b/pkg/sql/scan_test.go index 62451d1a62ab..b8cd7acf02ee 100644 --- a/pkg/sql/scan_test.go +++ b/pkg/sql/scan_test.go @@ -20,7 +20,7 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/sql/row" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -142,16 +142,21 @@ func TestScanBatches(t *testing.T) { // The table will have one key for the even rows, and two keys for the odd rows. numKeys := 3 * numAs * numBs / 2 - batchSizes := []int{1, 2, 5, 13, 100, numKeys - 1, numKeys, numKeys + 1} + batchByteSizes := []int{1, 2, 5, 13, 100, numKeys, numKeys * 5, numKeys * 100} - for _, batch := range batchSizes { + for _, batchBytesSize := range batchByteSizes { // We must set up a separate server for each batch size, as we cannot change - // it while the server is running (#53002). - t.Run(fmt.Sprintf("%d", batch), func(t *testing.T) { - restore := row.TestingSetKVBatchSize(int64(batch)) - defer restore() + // it while the server is running. + t.Run(fmt.Sprintf("%d", batchBytesSize), func(t *testing.T) { s, db, _ := serverutils.StartServer( - t, base.TestServerArgs{UseDatabase: "test"}) + t, base.TestServerArgs{ + UseDatabase: "test", + Knobs: base.TestingKnobs{ + DistSQL: &execinfra.TestingKnobs{ + TableReaderBatchBytesLimit: int64(batchBytesSize), + }, + }, + }) defer s.Stopper().Stop(context.Background()) if _, err := db.Exec(schema); err != nil { diff --git a/pkg/sql/stats/BUILD.bazel b/pkg/sql/stats/BUILD.bazel index 5feff480bc48..17301e8d082f 100644 --- a/pkg/sql/stats/BUILD.bazel +++ b/pkg/sql/stats/BUILD.bazel @@ -88,7 +88,7 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/tabledesc", - "//pkg/sql/row", + "//pkg/sql/execinfra", "//pkg/sql/rowenc", "//pkg/sql/rowexec", "//pkg/sql/sem/tree", diff --git a/pkg/sql/stats/create_stats_job_test.go b/pkg/sql/stats/create_stats_job_test.go index d3b2f6009dd6..e6e9cbb7cb5d 100644 --- a/pkg/sql/stats/create_stats_job_test.go +++ b/pkg/sql/stats/create_stats_job_test.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/row" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" @@ -299,15 +299,17 @@ func TestCreateStatsProgress(t *testing.T) { }(rowexec.SamplerProgressInterval) rowexec.SamplerProgressInterval = 10 - resetKVBatchSize := row.TestingSetKVBatchSize(10) - defer resetKVBatchSize() - var allowRequest chan struct{} var serverArgs base.TestServerArgs params := base.TestClusterArgs{ServerArgs: serverArgs} params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: createStatsRequestFilter(&allowRequest), } + params.ServerArgs.Knobs.DistSQL = &execinfra.TestingKnobs{ + // Force the stats job to iterate through the input rows instead of reading + // them all at once. + TableReaderBatchBytesLimit: 100, + } ctx := context.Background() const nodes = 1 @@ -348,7 +350,11 @@ func TestCreateStatsProgress(t *testing.T) { select { case allowRequest <- struct{}{}: case err := <-errCh: - t.Fatal(err) + if err == nil { + t.Fatalf("query unexpectedly finished") + } else { + t.Fatal(err) + } } } @@ -400,7 +406,11 @@ func TestCreateStatsProgress(t *testing.T) { select { case allowRequest <- struct{}{}: case err := <-errCh: - t.Fatal(err) + if err == nil { + t.Fatalf("query unexpectedly finished") + } else { + t.Fatal(err) + } } } diff --git a/pkg/sql/tablewriter_delete.go b/pkg/sql/tablewriter_delete.go index 662033b7276e..57f11e056116 100644 --- a/pkg/sql/tablewriter_delete.go +++ b/pkg/sql/tablewriter_delete.go @@ -147,7 +147,7 @@ func (td *tableDeleter) deleteAllRowsScan( return resume, err } if err := rf.StartScan( - ctx, td.txn, roachpb.Spans{resume}, true /* limit batches */, 0, traceKV, td.forceProductionBatchSizes, + ctx, td.txn, roachpb.Spans{resume}, row.DefaultBatchBytesLimit, row.NoRowLimit, traceKV, td.forceProductionBatchSizes, ); err != nil { return resume, err } @@ -284,7 +284,7 @@ func (td *tableDeleter) deleteIndexScan( return resume, err } if err := rf.StartScan( - ctx, td.txn, roachpb.Spans{resume}, true /* limit batches */, 0, traceKV, td.forceProductionBatchSizes, + ctx, td.txn, roachpb.Spans{resume}, row.DefaultBatchBytesLimit, row.NoRowLimit, traceKV, td.forceProductionBatchSizes, ); err != nil { return resume, err } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 4b6f0b874bca..78284b250a2c 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1630,6 +1630,7 @@ func TestLint(t *testing.T) { stream.GrepNot("pkg/sql/types/types.go.* var Uuid should be UUID"), stream.GrepNot("pkg/sql/oidext/oidext.go.*don't use underscores in Go names; const T_"), stream.GrepNot("server/api_v2.go.*package comment should be of the form"), + stream.GrepNot("type name will be used as row.RowLimit by other packages, and that stutters; consider calling this Limit"), ), func(s string) { t.Errorf("\n%s", s) }); err != nil { diff --git a/pkg/util/tracing/test_utils.go b/pkg/util/tracing/test_utils.go index 313b9da5391d..ae3d67abaa46 100644 --- a/pkg/util/tracing/test_utils.go +++ b/pkg/util/tracing/test_utils.go @@ -30,7 +30,7 @@ func FindMsgInRecording(recording Recording, msg string) int { // LogsContainMsg returns true if a Span's logs contain the given message. func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { for _, l := range sp.Logs { - // NOTE: With out logs, each LogRecord has a single field ("event") and + // NOTE: With our logs, each LogRecord has a single field ("event") and // value. for _, f := range l.Fields { if strings.Contains(f.Value, msg) { @@ -40,3 +40,18 @@ func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { } return false } + +// CountLogMessages counts the messages containing msg. +func CountLogMessages(sp tracingpb.RecordedSpan, msg string) int { + res := 0 + for _, l := range sp.Logs { + // NOTE: With our logs, each LogRecord has a single field ("event") and + // value. + for _, f := range l.Fields { + if strings.Contains(f.Value, msg) { + res++ + } + } + } + return res +}