From 90054c6b7b9629aeb904ffd4c0b9161f70e214b7 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Thu, 17 Feb 2022 20:13:08 -0500 Subject: [PATCH 1/3] spanconfig: limit # of tenant span configs Fixes #70555. In order to limit the number of span configs a tenant's able to install, we introduce a tenant-side spanconfig.Limiter. It presents the following interface: // Limiter is used to limit the number of span configs installed by // secondary tenants. It takes in a delta (typically the difference // in span configs between the committed and uncommitted state in // the txn), uses it to maintain an aggregate counter, and informs // the caller if exceeding the prescribed limit. type Limiter interface { ShouldLimit( ctx context.Context, txn *kv.Txn, delta int, ) (bool, error) } The delta is computed using a static helper, spanconfig.Delta: // Delta considers both the committed and uncommitted state of a // table descriptor and computes the difference in the number of // spans we can apply a configuration over. func Delta( ctx context.Context, s Splitter, committed, uncommitted catalog.TableDescriptor, ) (int, error) This limiter only applies to secondary tenants. The counter is maintained in a newly introduced (tenant-only) system table, using the following schema: CREATE TABLE system.span_count ( singleton BOOL DEFAULT TRUE, span_count INT NOT NULL, CONSTRAINT "primary" PRIMARY KEY (singleton), CONSTRAINT single_row CHECK (singleton), FAMILY "primary" (singleton, span_count) ); We need just two integration points for spanconfig.Limiter: - Right above CheckTwoVersionInvariant, where we're able to hook into the committed and to-be-committed descriptor state before txn commit; - In the GC job, when gc-ing table state. We decrement a table's split count when GC-ing the table for good. The per-tenant span config limit used is controlled by a new tenant read-only cluster setting: spanconfig.tenant_limit. Multi-tenant cluster settings (#73857) provides the infrastructure for the host tenant to be able to control this setting cluster wide, or to target a specific tenant at a time. We also need a migration here, to start tracking span counts for clusters with pre-existing tenants. We introduce a migration that scans over all table descriptors and seeds system.span_count with the right value. Given cluster version gates disseminate asynchronously, we also need a preliminary version to start tracking incremental changes. It's useful to introduce the notion of debt. This will be handy if/when we lower per-tenant limits, and also in the migration above since it's possible for pre-existing tenants to have committed state in violation of the prescribed limit. When in debt, schema changes that add new splits will be rejected (dropping tables/indexes/partitions/etc. will work just fine). When attempting a txn that goes over the configured limit, the UX is as follows: > CREATE TABLE db.t42(i INT PRIMARY KEY); pq: exceeded limit for number of table spans Release note: None Release justification: low risk, high benefit change Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/BUILD.bazel | 1 + pkg/ccl/backupccl/system_schema.go | 3 + .../migrationccl/migrationsccl/BUILD.bazel | 3 + .../seed_span_counts_external_test.go | 252 ++++++++++++++++++ .../testdata/multitenant | 1 + .../spanconfiglimiterccl/BUILD.bazel | 37 +++ .../spanconfiglimiterccl/datadriven_test.go | 147 ++++++++++ .../spanconfiglimiterccl/drop_table_test.go | 82 ++++++ .../spanconfiglimiterccl/main_test.go | 33 +++ .../spanconfiglimiterccl/testdata/indexes | 56 ++++ .../spanconfiglimiterccl/testdata/limit | 51 ++++ .../testdata/partitioning | 85 ++++++ .../spanconfiglimiterccl/testdata/tables | 55 ++++ .../spanconfiglimiterccl/testdata/unlimited | 19 ++ .../testdata/multitenant/basic | 3 + .../testdata/multitenant/protectedts | 1 + .../testdata/tenant/full_translate | 2 + pkg/clusterversion/cockroach_versions.go | 25 +- pkg/clusterversion/key_string.go | 7 +- pkg/migration/migrationjob/migration_job.go | 1 + pkg/migration/migrations/BUILD.bazel | 1 + pkg/migration/migrations/helpers_test.go | 2 +- pkg/migration/migrations/migrations.go | 12 + pkg/migration/migrations/span_count_table.go | 97 +++++++ pkg/migration/tenant_migration.go | 1 + pkg/server/BUILD.bazel | 2 + pkg/server/server_sql.go | 31 ++- pkg/spanconfig/BUILD.bazel | 2 + pkg/spanconfig/spanconfig.go | 45 ++++ pkg/spanconfig/spanconfiglimiter/BUILD.bazel | 23 ++ pkg/spanconfig/spanconfiglimiter/limiter.go | 99 +++++++ pkg/spanconfig/spanconfiglimiter/noop.go | 28 ++ pkg/spanconfig/spanconfigsplitter/BUILD.bazel | 5 +- pkg/spanconfig/spanconfigsplitter/noop.go | 28 ++ pkg/spanconfig/spanconfigsplitter/splitter.go | 10 + .../spanconfigtestcluster/tenant_state.go | 7 + pkg/spanconfig/testing_knobs.go | 4 + pkg/sql/alter_column_type.go | 4 +- pkg/sql/alter_index.go | 2 +- pkg/sql/alter_primary_key.go | 2 +- pkg/sql/alter_table.go | 2 +- pkg/sql/alter_table_locality.go | 2 +- pkg/sql/backfill.go | 9 +- pkg/sql/catalog/bootstrap/metadata.go | 2 + pkg/sql/catalog/catconstants/constants.go | 1 + pkg/sql/catalog/catprivilege/system.go | 1 + pkg/sql/catalog/descs/BUILD.bazel | 1 + pkg/sql/catalog/descs/factory.go | 31 ++- pkg/sql/catalog/descs/txn.go | 50 ++++ pkg/sql/catalog/systemschema/system.go | 39 +++ pkg/sql/catalog/tabledesc/safe_format_test.go | 4 +- pkg/sql/catalog/tabledesc/structured.go | 41 ++- pkg/sql/catalog/tabledesc/table_desc.go | 3 +- .../catalog/tabledesc/table_desc_builder.go | 2 +- pkg/sql/catalog/tabledesc/table_desc_test.go | 6 + pkg/sql/conn_executor_exec.go | 11 + pkg/sql/create_index.go | 2 +- pkg/sql/create_view.go | 2 +- pkg/sql/drop_index.go | 4 +- pkg/sql/exec_util.go | 7 + pkg/sql/gcjob/table_garbage_collection.go | 14 + pkg/sql/logictest/testdata/logic_test/system | 12 + .../testdata/logic_test/system_namespace | 1 + pkg/sql/refresh_materialized_view.go | 2 +- pkg/sql/schema_changer.go | 4 +- pkg/sql/table.go | 2 +- pkg/sql/tests/system_table_test.go | 2 +- pkg/sql/tests/testdata/initial_keys | 8 +- 70 files changed, 1476 insertions(+), 62 deletions(-) create mode 100644 pkg/ccl/migrationccl/migrationsccl/seed_span_counts_external_test.go create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/main_test.go create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/indexes create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/limit create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/partitioning create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/tables create mode 100644 pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/unlimited create mode 100644 pkg/migration/migrations/span_count_table.go create mode 100644 pkg/spanconfig/spanconfiglimiter/BUILD.bazel create mode 100644 pkg/spanconfig/spanconfiglimiter/limiter.go create mode 100644 pkg/spanconfig/spanconfiglimiter/noop.go create mode 100644 pkg/spanconfig/spanconfigsplitter/noop.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 7a413f173243..a3a0bb771cfa 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -193,4 +193,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 21.2-106 set the active cluster version in the format '.' +version version 21.2-112 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 3eea32444e55..ffb5fe6e365f 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -209,6 +209,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-106set the active cluster version in the format '.' +versionversion21.2-112set the active cluster version in the format '.' diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index f3b3dd24e70c..6cfecc05e749 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -37,6 +37,7 @@ ALL_TESTS = [ "//pkg/ccl/serverccl/statusccl:statusccl_test", "//pkg/ccl/serverccl:serverccl_test", "//pkg/ccl/spanconfigccl/spanconfigcomparedccl:spanconfigcomparedccl_test", + "//pkg/ccl/spanconfigccl/spanconfiglimiterccl:spanconfiglimiterccl_test", "//pkg/ccl/spanconfigccl/spanconfigreconcilerccl:spanconfigreconcilerccl_test", "//pkg/ccl/spanconfigccl/spanconfigsplitterccl:spanconfigsplitterccl_test", "//pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl:spanconfigsqltranslatorccl_test", diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 530154b9b711..6862d7453a3e 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -399,6 +399,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ shouldIncludeInClusterBackup: optInToClusterBackup, customRestoreFunc: tenantSettingsTableRestoreFunc, }, + systemschema.SpanCountTable.GetName(): { + shouldIncludeInClusterBackup: optOutOfClusterBackup, + }, } // GetSystemTablesToIncludeInClusterBackup returns a set of system table names that diff --git a/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel b/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel index 0b721fbb3ed1..eeadabe3b3eb 100644 --- a/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel +++ b/pkg/ccl/migrationccl/migrationsccl/BUILD.bazel @@ -4,6 +4,7 @@ go_test( name = "migrationsccl_test", srcs = [ "main_test.go", + "seed_span_counts_external_test.go", "seed_tenant_span_configs_external_test.go", ], deps = [ @@ -15,7 +16,9 @@ go_test( "//pkg/security", "//pkg/security/securitytest", "//pkg/server", + "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", diff --git a/pkg/ccl/migrationccl/migrationsccl/seed_span_counts_external_test.go b/pkg/ccl/migrationccl/migrationsccl/seed_span_counts_external_test.go new file mode 100644 index 000000000000..7e6bea82140d --- /dev/null +++ b/pkg/ccl/migrationccl/migrationsccl/seed_span_counts_external_test.go @@ -0,0 +1,252 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package migrationsccl_test + +import ( + "context" + gosql "database/sql" + "net/url" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// TestPreSeedSpanCountTable tests that incremental schema changes after +// PreSeedSpanCountTable is enabled get tracked as such. It also tests that once +// SeedSpanCountTable is reached, the span count is updated to capture the most +// up-to-date view of all schema objects. Specifically, we're not +// double-counting the incremental update we tracked in the +// PreSeedSpanCountTable state. +func TestPreSeedSpanCountTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + v0 = clusterversion.ByKey(clusterversion.SpanCountTable) + v1 = clusterversion.ByKey(clusterversion.PreSeedSpanCountTable) + v2 = clusterversion.ByKey(clusterversion.SeedSpanCountTable) + ) + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, + }, + }, + }) + + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0) + hostDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + tenantID := roachpb.MakeTenantID(10) + tenantSettings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &tenantSettings.SV)) + tenant, err := ts.StartTenant(ctx, base.TestTenantArgs{ + TenantID: tenantID, + TestingKnobs: base.TestingKnobs{}, + Settings: tenantSettings, + }) + require.NoError(t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(t, tenant.SQLAddr(), "Tenant", url.User(security.RootUser)) + defer cleanupPGUrl() + + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + require.NoError(t, err) + defer func() { require.NoError(t, tenantSQLDB.Close()) }() + + // Upgrade the host cluster all the way. + hostDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + + var spanCount, numRows int + tenantDB := sqlutils.MakeSQLRunner(tenantSQLDB) + + tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v0.String()}}) + tenantDB.Exec(t, `CREATE TABLE t(k INT PRIMARY KEY)`) + tenantDB.QueryRow(t, `SELECT count(*) FROM system.span_count`).Scan(&numRows) + require.Equal(t, 0, numRows) + + tenantDB.Exec(t, "SET CLUSTER SETTING version = $1", v1.String()) + tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v1.String()}}) + tenantDB.Exec(t, `CREATE INDEX idx ON t (k)`) + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 2, spanCount) + + tenantDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v2.String()}}) + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 5, spanCount) +} + +// TestSeedSpanCountTable tests that the migration seeds system.span_count +// correctly for secondary tenants. +func TestSeedSpanCountTable(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + v0 = clusterversion.ByKey(clusterversion.SpanCountTable) + v2 = clusterversion.ByKey(clusterversion.SeedSpanCountTable) + ) + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, + }, + }, + }) + + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0) + hostDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + tenantID := roachpb.MakeTenantID(10) + tenantSettings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &tenantSettings.SV)) + tenant, err := ts.StartTenant(ctx, base.TestTenantArgs{ + TenantID: tenantID, + TestingKnobs: base.TestingKnobs{}, + Settings: tenantSettings, + }) + require.NoError(t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(t, tenant.SQLAddr(), "Tenant", url.User(security.RootUser)) + defer cleanupPGUrl() + + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + require.NoError(t, err) + defer func() { require.NoError(t, tenantSQLDB.Close()) }() + + tenantDB := sqlutils.MakeSQLRunner(tenantSQLDB) + tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v0.String()}}) + + // Upgrade the host cluster. + hostDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{v0.String()}}) + + tenantDB.Exec(t, `CREATE TABLE t(k INT PRIMARY KEY)`) + + var spanCount, numRows int + tenantDB.QueryRow(t, `SELECT count(*) FROM system.span_count`).Scan(&numRows) + require.Equal(t, 0, numRows) + + tenantDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 3, spanCount) +} + +// TestSeedSpanCountTableOverLimit tests that the migration seeds +// system.span_count correctly for secondary tenants, even if over the +// proscribed limit. In these cases the tenant goes into debt -- all subsequent +// schema changes that add schema elements will be rejected. Attempts to free up +// spans however will be accepted. +func TestSeedSpanCountTableOverLimit(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var ( + v0 = clusterversion.ByKey(clusterversion.SpanCountTable) + v2 = clusterversion.ByKey(clusterversion.SeedSpanCountTable) + ) + + ctx := context.Background() + settings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &settings.SV)) + + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: v0, + }, + }, + }, + }) + + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0) + hostDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + const limit = 1 + tenantID := roachpb.MakeTenantID(10) + tenantSettings := cluster.MakeTestingClusterSettingsWithVersions(v2, v0, false /* initializeVersion */) + require.NoError(t, clusterversion.Initialize(ctx, v0, &tenantSettings.SV)) + tenant, err := ts.StartTenant(ctx, base.TestTenantArgs{ + TenantID: tenantID, + TestingKnobs: base.TestingKnobs{ + SpanConfig: &spanconfig.TestingKnobs{ + ExcludeDroppedDescriptorsFromLookup: true, + LimiterLimitOverride: func() int64 { + return limit + }, + }, + }, + Settings: tenantSettings, + }) + require.NoError(t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(t, tenant.SQLAddr(), "Tenant", url.User(security.RootUser)) + defer cleanupPGUrl() + + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + require.NoError(t, err) + defer func() { require.NoError(t, tenantSQLDB.Close()) }() + + // Upgrade the host cluster. + hostDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + + tenantDB := sqlutils.MakeSQLRunner(tenantSQLDB) + tenantDB.Exec(t, `CREATE TABLE t1(k INT PRIMARY KEY)`) + tenantDB.Exec(t, `CREATE TABLE t2(k INT PRIMARY KEY)`) + tenantDB.Exec(t, `CREATE TABLE t3(k INT PRIMARY KEY)`) + + var spanCount int + tenantDB.Exec(t, "SET CLUSTER SETTING version = $1", v2.String()) + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 9, spanCount) + + _, err = tenantDB.DB.ExecContext(ctx, `CREATE TABLE t4(k INT PRIMARY KEY)`) + require.True(t, testutils.IsError(err, "exceeded limit for number of table spans")) + + tenantDB.Exec(t, `DROP TABLE t3`) + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 6, spanCount) +} diff --git a/pkg/ccl/spanconfigccl/spanconfigcomparedccl/testdata/multitenant b/pkg/ccl/spanconfigccl/spanconfigcomparedccl/testdata/multitenant index 9c8f237b7874..b3d79b2f7265 100644 --- a/pkg/ccl/spanconfigccl/spanconfigcomparedccl/testdata/multitenant +++ b/pkg/ccl/spanconfigccl/spanconfigcomparedccl/testdata/multitenant @@ -117,6 +117,7 @@ diff offset=48 +/Tenant/11/Table/43 database system (tenant) +/Tenant/11/Table/44 database system (tenant) +/Tenant/11/Table/46 database system (tenant) ++/Tenant/11/Table/50 database system (tenant) +/Tenant/11/Table/106 ttl_seconds=1000 num_replicas=42 +/Tenant/11/Table/107 range default diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel new file mode 100644 index 000000000000..128318644767 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel @@ -0,0 +1,37 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "spanconfiglimiterccl_test", + srcs = [ + "datadriven_test.go", + "drop_table_test.go", + "main_test.go", + ], + data = glob(["testdata/**"]), + deps = [ + "//pkg/base", + "//pkg/ccl/kvccl/kvtenantccl", + "//pkg/ccl/partitionccl", + "//pkg/ccl/utilccl", + "//pkg/config", + "//pkg/config/zonepb", + "//pkg/keys", + "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster", + "//pkg/sql/gcjob", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/randutil", + "@com_github_cockroachdb_datadriven//:datadriven", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go new file mode 100644 index 000000000000..44d2922665ca --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/datadriven_test.go @@ -0,0 +1,147 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package spanconfiglimiterccl + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// TestDataDriven is a data-driven test for spanconfig.Limiter. It offers the +// following commands: +// +// - "initialize" tenant= +// Initialize a secondary tenant with the given ID. +// +// - "exec-sql" [tenant=] +// Executes the input SQL query for the given tenant. All statements are +// executed in a single transaction. +// +// - "query-sql" [tenant=] [retry] +// Executes the input SQL query for the given tenant and print the results. +// If retry is specified and the expected results do not match the actual +// results, the query will be retried under a testutils.SucceedsSoon block. +// If run with -rewrite, we insert a 500ms sleep before executing the query +// once. +// +// - override limit= +// Override the span limit each tenant is configured with. +// +func TestDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + // TODO(irfansharif): This is a stop-gap for tenant read-only cluster + // settings. See https://github.com/cockroachdb/cockroach/pull/76929. Once + // that's done, this test should be updated to use: + // SET CLUSTER SETTING spanconfig.tenant_limit = + limitOverride := 50 + scKnobs := &spanconfig.TestingKnobs{ + // Instead of relying on the GC job to wait out TTLs and clear out + // descriptors, let's simply exclude dropped tables to simulate + // descriptors no longer existing. + ExcludeDroppedDescriptorsFromLookup: true, + LimiterLimitOverride: func() int64 { + return int64(limitOverride) + }, + } + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + SpanConfig: scKnobs, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + { + tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '20ms'`) + } + + spanConfigTestCluster := spanconfigtestcluster.NewHandle(t, tc, scKnobs) + defer spanConfigTestCluster.Cleanup() + + spanConfigTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID) + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + tenantID := roachpb.SystemTenantID + if d.HasArg("tenant") { + var id uint64 + d.ScanArgs(t, "tenant", &id) + tenantID = roachpb.MakeTenantID(id) + } + + tenant, found := spanConfigTestCluster.LookupTenant(tenantID) + if d.Cmd != "initialize" { + require.Truef(t, found, "tenant %s not found (was it initialized?)", tenantID) + } + + switch d.Cmd { + case "initialize": + spanConfigTestCluster.InitializeTenant(ctx, tenantID) + + case "exec-sql": + if err := tenant.ExecWithErr(d.Input); err != nil { + return fmt.Sprintf("err: %s", err) + } + + case "query-sql": + query := func() string { + rows := tenant.Query(d.Input) + output, err := sqlutils.RowsToDataDrivenOutput(rows) + require.NoError(t, err) + return output + } + if !d.HasArg("retry") { + return query() + } + + if d.Rewrite { + time.Sleep(500 * time.Millisecond) + return query() + } + + var output string + testutils.SucceedsSoon(t, func() error { + if output = query(); output != d.Expected { + return errors.Newf("expected %q, got %q; retrying..", d.Expected, output) + } + return nil + }) + return output + + case "override": + d.ScanArgs(t, "limit", &limitOverride) + + default: + t.Fatalf("unknown command: %s", d.Cmd) + } + + return "" + }) + }) +} diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go new file mode 100644 index 000000000000..172c73213ad0 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go @@ -0,0 +1,82 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package spanconfiglimiterccl + +import ( + "context" + gosql "database/sql" + "net/url" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/config" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/gcjob" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestDropTableLowersSpanCount(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + defer gcjob.SetSmallMaxGCIntervalForTest()() + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + + defer tc.Stopper().Stop(ctx) + ts := tc.Server(0) + + tenantID := roachpb.MakeTenantID(10) + tenant, err := ts.StartTenant(ctx, base.TestTenantArgs{ + TenantID: tenantID, + TestingKnobs: base.TestingKnobs{}, + }) + require.NoError(t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(t, tenant.SQLAddr(), "Tenant", url.User(security.RootUser)) + defer cleanupPGUrl() + + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + + zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig.GC.TTLSeconds = 1 + config.TestingSetupZoneConfigHook(tc.Stopper()) + // TODO(irfansharif): Work around for #75864. + config.TestingSetZoneConfig(keys.TenantsRangesID, zoneConfig) + + require.NoError(t, err) + defer func() { require.NoError(t, tenantSQLDB.Close()) }() + + tenantDB := sqlutils.MakeSQLRunner(tenantSQLDB) + tenantDB.Exec(t, `CREATE TABLE t(k INT PRIMARY KEY)`) + + var spanCount int + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + require.Equal(t, 3, spanCount) + + tenantDB.Exec(t, `DROP TABLE t`) + + testutils.SucceedsSoon(t, func() error { + tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount) + if spanCount != 0 { + return errors.Newf("expected zero span count, found %d", spanCount) + } + return nil + }) +} diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/main_test.go b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/main_test.go new file mode 100644 index 000000000000..da37b69224ff --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package spanconfiglimiterccl + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go + +func TestMain(m *testing.M) { + defer utilccl.TestingEnableEnterprise()() + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/indexes b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/indexes new file mode 100644 index 000000000000..9ed6a061b0db --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/indexes @@ -0,0 +1,56 @@ +# Ensure that system.span_count is maintained appropriately when creating and +# dropping secondary indexes, and then dropping the table entirely. + +initialize tenant=10 +---- + +query-sql tenant=10 +SELECT count(*) FROM system.span_count; +---- +0 + +exec-sql tenant=10 +CREATE DATABASE db; +CREATE TABLE db.t(i INT PRIMARY KEY, j INT); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +exec-sql tenant=10 +CREATE INDEX idx2 ON db.t (j); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +5 + +exec-sql tenant=10 +DROP INDEX db.t@idx2; +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +exec-sql tenant=10 +CREATE INDEX idx4 ON db.t (j); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +5 + +exec-sql tenant=10 +DROP TABLE db.t; +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +0 diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/limit b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/limit new file mode 100644 index 000000000000..3d1999cff806 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/limit @@ -0,0 +1,51 @@ +# Ensure that we respect tenant span config limits, rejecting schema change +# operations that take us past it. + +initialize tenant=10 +---- + +exec-sql tenant=10 +CREATE DATABASE db; +CREATE TABLE db.t1(i INT PRIMARY KEY); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +override limit=3 +---- + +exec-sql tenant=10 +CREATE TABLE db.t2(i INT PRIMARY KEY); +---- +err: pq: exceeded limit for number of table spans + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +query-sql tenant=10 +SELECT table_name FROM [SHOW TABLES FROM db]; +---- +t1 + +exec-sql tenant=10 +DROP TABLE db.t1; +---- + +exec-sql tenant=10 +CREATE TABLE db.t2(i INT PRIMARY KEY); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +query-sql tenant=10 +SELECT table_name FROM [SHOW TABLES FROM db]; +---- +t2 diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/partitioning b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/partitioning new file mode 100644 index 000000000000..3514610c17d0 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/partitioning @@ -0,0 +1,85 @@ +# Ensure that we respect tenant span config limits, rejecting partitioning +# changes operations that would take us past it. Repartitioning old partitions, +# or dropping them entirely, should return quota back for subsequent use. + +initialize tenant=10 +---- + +exec-sql tenant=10 +CREATE DATABASE db; +CREATE TABLE db.list_partitions(i INT PRIMARY KEY, j INT, k INT); +CREATE INDEX idx_i ON db.list_partitions (i); +CREATE INDEX idx_j ON db.list_partitions (j); +CREATE INDEX idx_k ON db.list_partitions (k); +ALTER INDEX db.list_partitions@idx_i PARTITION BY LIST (i) ( + PARTITION one_and_five VALUES IN (1, 5), + PARTITION everything_else VALUES IN (DEFAULT) +); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +15 + +override limit=15 +---- + +exec-sql tenant=10 +ALTER INDEX db.list_partitions@idx_j PARTITION BY LIST (j) ( + PARTITION one_and_five VALUES IN (1, 5), + PARTITION everything_else VALUES IN (DEFAULT) +); +---- +err: pq: exceeded limit for number of table spans + +# Drop partitioning spans, expect to see span_count reduce accordingly. +# +exec-sql tenant=10 +ALTER INDEX db.list_partitions@idx_i PARTITION BY NOTHING +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +9 + +# Re-attempt the secondary index partition, should succeed. +# +exec-sql tenant=10 +ALTER INDEX db.list_partitions@idx_j PARTITION BY LIST (j) ( + PARTITION one_and_five VALUES IN (1, 5), + PARTITION everything_else VALUES IN (DEFAULT) +); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +15 + +# We could also claw back quota by dropping a partitioned index entirely. +# +exec-sql tenant=10 +ALTER INDEX db.list_partitions@idx_k PARTITION BY LIST (k) ( + PARTITION one_and_five VALUES IN (1, 5), + PARTITION everything_else VALUES IN (DEFAULT) +); +---- +err: pq: exceeded limit for number of table spans + +exec-sql tenant=10 +DROP INDEX db.list_partitions@idx_j; +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +7 + +exec-sql tenant=10 +ALTER INDEX db.list_partitions@idx_k PARTITION BY LIST (k) ( + PARTITION one_and_five VALUES IN (1, 5), + PARTITION everything_else VALUES IN (DEFAULT) +); +---- diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/tables b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/tables new file mode 100644 index 000000000000..8d17e44f239c --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/tables @@ -0,0 +1,55 @@ +# Ensure that system.span_count is maintained appropriately when creating and +# dropping tables. + +initialize tenant=10 +---- + +query-sql tenant=10 +SELECT count(*) FROM system.span_count; +---- +0 + +exec-sql tenant=10 +CREATE DATABASE db; +CREATE TABLE db.t1(i INT PRIMARY KEY); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +3 + +exec-sql tenant=10 +CREATE TABLE db.t2(i INT PRIMARY KEY); +CREATE TABLE db.t3(i INT PRIMARY KEY); +CREATE TABLE db.t4(i INT PRIMARY KEY); +CREATE TABLE db.t5(i INT PRIMARY KEY); +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +15 + +exec-sql tenant=10 +DROP TABLE db.t1; +---- + +exec-sql tenant=10 +DROP TABLE db.t2; +DROP TABLE db.t3; +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +6 + +exec-sql tenant=10 +DROP DATABASE db CASCADE; +---- + +query-sql tenant=10 +SELECT span_count FROM system.span_count; +---- +0 diff --git a/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/unlimited b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/unlimited new file mode 100644 index 000000000000..309ff26a3121 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfiglimiterccl/testdata/unlimited @@ -0,0 +1,19 @@ +# Ensure that the system tenant entirely ignores span config limits. + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(i INT PRIMARY KEY); +---- + +override limit=3 +---- + +exec-sql +CREATE TABLE db.t2(i INT PRIMARY KEY); +---- + +query-sql +SELECT table_name FROM [SHOW TABLES FROM db]; +---- +t1 +t2 diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic index 40d4562c566d..2369e373f5e1 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/basic @@ -67,6 +67,7 @@ upsert /Tenant/10/Table/4{2-3} database system (tenant) upsert /Tenant/10/Table/4{3-4} database system (tenant) upsert /Tenant/10/Table/4{4-5} database system (tenant) upsert /Tenant/10/Table/4{6-7} database system (tenant) +upsert /Tenant/10/Table/5{0-1} database system (tenant) state offset=47 ---- @@ -106,6 +107,7 @@ state offset=47 /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/5{0-1} database system (tenant) /Tenant/11{-"\x00"} database system (tenant) exec-sql tenant=10 @@ -130,6 +132,7 @@ state offset=81 ---- ... /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/10{6-7} range default /Tenant/10/Table/10{7-8} range default /Tenant/10/Table/11{2-3} range default diff --git a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts index f92e008367b1..9105b88efa33 100644 --- a/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigreconcilerccl/testdata/multitenant/protectedts @@ -83,6 +83,7 @@ upsert /Tenant/10/Table/4{2-3} database system (tenant) upsert /Tenant/10/Table/4{3-4} database system (tenant) upsert /Tenant/10/Table/4{4-5} database system (tenant) upsert /Tenant/10/Table/4{6-7} database system (tenant) +upsert /Tenant/10/Table/5{0-1} database system (tenant) exec-sql tenant=10 CREATE DATABASE db; diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate index efcaed411bee..cb2c73c90e76 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate @@ -51,6 +51,7 @@ full-translate /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/11{0-1} range default /Tenant/10/Table/11{1-2} range default /Tenant/10/Table/11{2-3} range default @@ -92,6 +93,7 @@ translate named-zone=default /Tenant/10/Table/4{3-4} database system (tenant) /Tenant/10/Table/4{4-5} database system (tenant) /Tenant/10/Table/4{6-7} database system (tenant) +/Tenant/10/Table/5{0-1} database system (tenant) /Tenant/10/Table/11{0-1} range default /Tenant/10/Table/11{1-2} range default /Tenant/10/Table/11{2-3} range default diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index ae7ae53b7cd1..d9c686b589b7 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -313,11 +313,9 @@ const ( // PebbleFormatSplitUserKeysMarked performs a Pebble-level migration and // upgrades the Pebble format major version to FormatSplitUserKeysMarked. PebbleFormatSplitUserKeysMarked - // IncrementalBackupSubdir enables backing up new incremental backups to a // dedicated subdirectory, to make it easier to apply a different ttl. IncrementalBackupSubdir - // DateStyleIntervalStyleCastRewrite rewrites cast that cause inconsistencies // when DateStyle/IntervalStyle is enabled. DateStyleIntervalStyleCastRewrite @@ -327,7 +325,6 @@ const ( // ClusterLocksVirtualTable enables querying the crdb_internal.cluster_locks // virtual table, which sends a QueryLocksRequest RPC to all cluster ranges. ClusterLocksVirtualTable - // AutoStatsTableSettings is the version where we allow auto stats related // table settings. AutoStatsTableSettings @@ -335,11 +332,19 @@ const ( ForecastStats // SuperRegions enables the usage on super regions. SuperRegions - // EnableNewChangefeedOptions enables the usage of new changefeed options // such as end_time, initial_scan_only, and setting the value of initial_scan // to 'yes|no|only' EnableNewChangefeedOptions + // SpanCountTable adds system.span_count to track the number of committed + // tenant spans. + SpanCountTable + // PreSeedSpanCountTable precedes PreSeedSpanCountTable, it enables span + // accounting for incremental schema changes. + PreSeedSpanCountTable + // SeedSpanCountTable seeds system.span_count with the number of committed + // tenant spans. + SeedSpanCountTable // ************************************************* // Step (1): Add new versions here. @@ -579,6 +584,18 @@ var versionsSingleton = keyedVersions{ Key: EnableNewChangefeedOptions, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 106}, }, + { + Key: SpanCountTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 108}, + }, + { + Key: PreSeedSpanCountTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 110}, + }, + { + Key: SeedSpanCountTable, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 112}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index c2ceb6c39abc..e16e61122cf9 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -59,11 +59,14 @@ func _() { _ = x[ForecastStats-48] _ = x[SuperRegions-49] _ = x[EnableNewChangefeedOptions-50] + _ = x[SpanCountTable-51] + _ = x[PreSeedSpanCountTable-52] + _ = x[SeedSpanCountTable-53] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptions" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTable" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314, 1328, 1349, 1367} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/migration/migrationjob/migration_job.go b/pkg/migration/migrationjob/migration_job.go index 672e86e90493..f924f27df94c 100644 --- a/pkg/migration/migrationjob/migration_job.go +++ b/pkg/migration/migrationjob/migration_job.go @@ -92,6 +92,7 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error { TestingKnobs: execCtx.ExecCfg().MigrationTestingKnobs, } tenantDeps.SpanConfig.KVAccessor = execCtx.ExecCfg().SpanConfigKVAccessor + tenantDeps.SpanConfig.Splitter = execCtx.ExecCfg().SpanConfigSplitter tenantDeps.SpanConfig.Default = execCtx.ExecCfg().DefaultZoneConfig.AsSpanConfig() err = m.Run(ctx, cv, tenantDeps, r.j) diff --git a/pkg/migration/migrations/BUILD.bazel b/pkg/migration/migrations/BUILD.bazel index 12b68055b2dc..8ec16e6dee32 100644 --- a/pkg/migration/migrations/BUILD.bazel +++ b/pkg/migration/migrations/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "remove_invalid_database_privileges.go", "schema_changes.go", "seed_tenant_span_configs.go", + "span_count_table.go", "tenant_settings.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrations", diff --git a/pkg/migration/migrations/helpers_test.go b/pkg/migration/migrations/helpers_test.go index 07ce34bfaf2e..59dd1dedd667 100644 --- a/pkg/migration/migrations/helpers_test.go +++ b/pkg/migration/migrations/helpers_test.go @@ -84,7 +84,7 @@ func InjectLegacyTable( return err } tab.TableDescriptor = builder.BuildCreatedMutableTable().TableDescriptor - tab.Version = tab.ClusterVersion.Version + 1 + tab.Version = tab.ClusterVersion().Version + 1 return descriptors.WriteDesc(ctx, false /* kvTrace */, tab, txn) }) require.NoError(t, err) diff --git a/pkg/migration/migrations/migrations.go b/pkg/migration/migrations/migrations.go index 4f567579f54e..190bfc3561f6 100644 --- a/pkg/migration/migrations/migrations.go +++ b/pkg/migration/migrations/migrations.go @@ -131,6 +131,18 @@ var migrations = []migration.Migration{ NoPrecondition, fixCastForStyleMigration, ), + migration.NewTenantMigration( + "add the system.span_count table", + toCV(clusterversion.SpanCountTable), + NoPrecondition, + spanCountTableMigration, + ), + migration.NewTenantMigration( + "seed system.span_count with span count for existing tenants", + toCV(clusterversion.SeedSpanCountTable), + NoPrecondition, + seedSpanCountTableMigration, + ), } func init() { diff --git a/pkg/migration/migrations/span_count_table.go b/pkg/migration/migrations/span_count_table.go new file mode 100644 index 000000000000..8abdf96c6336 --- /dev/null +++ b/pkg/migration/migrations/span_count_table.go @@ -0,0 +1,97 @@ +// Copyright 2022 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 migrations + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/errors" +) + +// spanCountTableMigration creates the system.span_count table for secondary +// tenants. +func spanCountTableMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, +) error { + if d.Codec.ForSystemTenant() { + return nil // only applicable for secondary tenants + } + + return createSystemTable( + ctx, d.DB, d.Codec, systemschema.SpanCountTable, + ) +} + +// seedSpanCountTableMigration seeds system.span_count with data for existing +// secondary tenants. +func seedSpanCountTableMigration( + ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps, _ *jobs.Job, +) error { + if d.Codec.ForSystemTenant() { + return nil // only applicable for secondary tenants + } + + return d.CollectionFactory.Txn(ctx, d.InternalExecutor, d.DB, func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error { + dbs, err := descriptors.GetAllDatabaseDescriptors(ctx, txn) + if err != nil { + return err + } + + var spanCount int + for _, db := range dbs { + if db.GetID() == systemschema.SystemDB.GetID() { + continue // we don't count system table descriptors + } + + tables, err := descriptors.GetAllTableDescriptorsInDatabase(ctx, txn, db.GetID()) + if err != nil { + return err + } + + for _, table := range tables { + splits, err := d.SpanConfig.Splitter.Splits(ctx, table) + if err != nil { + return err + } + spanCount += splits + } + } + + const seedSpanCountStmt = ` +INSERT INTO system.span_count (span_count) VALUES ($1) +ON CONFLICT (singleton) +DO UPDATE SET span_count = $1 +RETURNING span_count +` + datums, err := d.InternalExecutor.QueryRowEx(ctx, "seed-span-count", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + seedSpanCountStmt, spanCount) + if err != nil { + return err + } + if len(datums) != 1 { + return errors.AssertionFailedf("expected to return 1 row, return %d", len(datums)) + } + if insertedSpanCount := int64(tree.MustBeDInt(datums[0])); insertedSpanCount != int64(spanCount) { + return errors.AssertionFailedf("expected to insert %d, got %d", spanCount, insertedSpanCount) + } + return nil + }) +} diff --git a/pkg/migration/tenant_migration.go b/pkg/migration/tenant_migration.go index 4c6ca923bf66..f0204e1349db 100644 --- a/pkg/migration/tenant_migration.go +++ b/pkg/migration/tenant_migration.go @@ -39,6 +39,7 @@ type TenantDeps struct { SpanConfig struct { // deps for span config migrations; can be removed accordingly spanconfig.KVAccessor + spanconfig.Splitter Default roachpb.SpanConfig } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 47dd9900e594..1278d83c9c53 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -138,9 +138,11 @@ go_library( "//pkg/spanconfig/spanconfigjob", "//pkg/spanconfig/spanconfigkvaccessor", "//pkg/spanconfig/spanconfigkvsubscriber", + "//pkg/spanconfig/spanconfiglimiter", "//pkg/spanconfig/spanconfigmanager", "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigreconciler", + "//pkg/spanconfig/spanconfigsplitter", "//pkg/spanconfig/spanconfigsqltranslator", "//pkg/spanconfig/spanconfigsqlwatcher", "//pkg/sql", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 43ee5c969184..b8d4503661b7 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -51,8 +51,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/tracedumper" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfiglimiter" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigmanager" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigreconciler" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsplitter" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqltranslator" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqlwatcher" "github.com/cockroachdb/cockroach/pkg/sql" @@ -552,11 +554,33 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { compactEngineSpanFunc = cli.CompactEngineSpan } + spanConfig := struct { + manager *spanconfigmanager.Manager + sqlTranslatorFactory *spanconfigsqltranslator.Factory + sqlWatcher *spanconfigsqlwatcher.SQLWatcher + splitter spanconfig.Splitter + limiter spanconfig.Limiter + }{} + if codec.ForSystemTenant() { + spanConfig.limiter = spanconfiglimiter.NoopLimiter{} + spanConfig.splitter = spanconfigsplitter.NoopSplitter{} + } else { + spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) + spanConfig.splitter = spanconfigsplitter.New(codec, spanConfigKnobs) + spanConfig.limiter = spanconfiglimiter.New( + cfg.circularInternalExecutor, + cfg.Settings, + spanConfigKnobs, + ) + } + collectionFactory := descs.NewCollectionFactory( cfg.Settings, leaseMgr, virtualSchemas, hydratedTablesCache, + spanConfig.splitter, + spanConfig.limiter, ) clusterIDForSQL := cfg.rpcContext.LogicalClusterID @@ -936,11 +960,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.MigrationTestingKnobs = knobs } - spanConfig := struct { - manager *spanconfigmanager.Manager - sqlTranslatorFactory *spanconfigsqltranslator.Factory - sqlWatcher *spanconfigsqlwatcher.SQLWatcher - }{} if !codec.ForSystemTenant() || !cfg.SpanConfigsDisabled { // Instantiate a span config manager. If we're the host tenant we'll // only do it unless COCKROACH_DISABLE_SPAN_CONFIGS is set. @@ -980,6 +999,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.SpanConfigReconciler = spanConfigReconciler } execCfg.SpanConfigKVAccessor = cfg.spanConfigAccessor + execCfg.SpanConfigLimiter = spanConfig.limiter + execCfg.SpanConfigSplitter = spanConfig.splitter temporaryObjectCleaner := sql.NewTemporaryObjectCleaner( cfg.Settings, diff --git a/pkg/spanconfig/BUILD.bazel b/pkg/spanconfig/BUILD.bazel index 19b4a19b8598..0fbf5adb9886 100644 --- a/pkg/spanconfig/BUILD.bazel +++ b/pkg/spanconfig/BUILD.bazel @@ -20,8 +20,10 @@ go_library( "//pkg/roachpb", "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/systemschema", "//pkg/util/encoding", "//pkg/util/hlc", + "//pkg/util/log", "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 5ba8758603fa..f1c326891da9 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -18,7 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" ) // KVAccessor mediates access to KV span configurations pertaining to a given @@ -257,6 +259,15 @@ type StoreReader interface { GetSpanConfigForKey(ctx context.Context, key roachpb.RKey) (roachpb.SpanConfig, error) } +// Limiter is used to limit the number of span configs installed by secondary +// tenants. It takes in a delta (typically the difference in span configs +// between the committed and uncommitted state in the txn), uses it to maintain +// an aggregate counter, and informs the caller if exceeding the prescribed +// limit. +type Limiter interface { + ShouldLimit(ctx context.Context, txn *kv.Txn, delta int) (bool, error) +} + // Splitter returns the number of split points for the given table descriptor. // It steps through every "unit" that we can apply configurations over (table, // indexes, partitions and sub-partitions) and figures out the actual key @@ -288,6 +299,40 @@ type Splitter interface { Splits(ctx context.Context, table catalog.TableDescriptor) (int, error) } +// Delta considers both the committed and uncommitted state of a table +// descriptor and computes the difference in the number of spans we can apply a +// configuration over. +func Delta( + ctx context.Context, s Splitter, committed, uncommitted catalog.TableDescriptor, +) (int, error) { + if committed == nil && uncommitted == nil { + log.Fatalf(ctx, "unexpected: got two nil table descriptors") + } + + var nonNilDesc catalog.TableDescriptor + if committed != nil { + nonNilDesc = committed + } else { + nonNilDesc = uncommitted + } + if nonNilDesc.GetParentID() == systemschema.SystemDB.GetID() { + return 0, nil // we don't count tables in the system database + } + + uncommittedSplits, err := s.Splits(ctx, uncommitted) + if err != nil { + return 0, err + } + + committedSplits, err := s.Splits(ctx, committed) + if err != nil { + return 0, err + } + + delta := uncommittedSplits - committedSplits + return delta, nil +} + // SQLUpdate captures either a descriptor or a protected timestamp update. // It is the unit emitted by the SQLWatcher. type SQLUpdate struct { diff --git a/pkg/spanconfig/spanconfiglimiter/BUILD.bazel b/pkg/spanconfig/spanconfiglimiter/BUILD.bazel new file mode 100644 index 000000000000..db8f3f60d9ac --- /dev/null +++ b/pkg/spanconfig/spanconfiglimiter/BUILD.bazel @@ -0,0 +1,23 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "spanconfiglimiter", + srcs = [ + "limiter.go", + "noop.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfiglimiter", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/kv", + "//pkg/security", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/spanconfig", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/spanconfig/spanconfiglimiter/limiter.go b/pkg/spanconfig/spanconfiglimiter/limiter.go new file mode 100644 index 000000000000..fba1c5f1d527 --- /dev/null +++ b/pkg/spanconfig/spanconfiglimiter/limiter.go @@ -0,0 +1,99 @@ +// Copyright 2022 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 spanconfiglimiter is used to limit how many span configs are +// installed by tenants. +package spanconfiglimiter + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/errors" +) + +var _ spanconfig.Limiter = &Limiter{} + +// tenantLimitSetting controls how many span configs a secondary tenant is +// allowed to install. It's settable only by the system tenant. +var tenantLimitSetting = settings.RegisterIntSetting( + settings.TenantReadOnly, + "spanconfig.tenant_limit", + "limit on the number of span configs a tenant is allowed to install", + 5000, +) + +// Limiter is used to limit the number of span configs installed by secondary +// tenants. It's a concrete implementation of the spanconfig.Limiter interface. +type Limiter struct { + ie sqlutil.InternalExecutor + settings *cluster.Settings + knobs *spanconfig.TestingKnobs +} + +// New constructs and returns a Limiter. +func New( + ie sqlutil.InternalExecutor, settings *cluster.Settings, knobs *spanconfig.TestingKnobs, +) *Limiter { + if knobs == nil { + knobs = &spanconfig.TestingKnobs{} + } + return &Limiter{ + ie: ie, + settings: settings, + knobs: knobs, + } +} + +// ShouldLimit is part of the spanconfig.Limiter interface. +func (l *Limiter) ShouldLimit(ctx context.Context, txn *kv.Txn, delta int) (bool, error) { + if !l.settings.Version.IsActive(ctx, clusterversion.PreSeedSpanCountTable) { + return false, nil // nothing to do + } + + if delta == 0 { + return false, nil + } + + limit := tenantLimitSetting.Get(&l.settings.SV) + if overrideFn := l.knobs.LimiterLimitOverride; overrideFn != nil { + limit = overrideFn() + } + + const updateSpanCountStmt = ` +INSERT INTO system.span_count (span_count) VALUES ($1) +ON CONFLICT (singleton) +DO UPDATE SET span_count = system.span_count.span_count + $1 +RETURNING span_count +` + datums, err := l.ie.QueryRowEx(ctx, "update-span-count", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + updateSpanCountStmt, delta) + if err != nil { + return false, err + } + if len(datums) != 1 { + return false, errors.AssertionFailedf("expected to return 1 row, return %d", len(datums)) + } + + if delta < 0 { + return false, nil // always allowed to lower span count + } + spanCountWithDelta := int64(tree.MustBeDInt(datums[0])) + return spanCountWithDelta > limit, nil +} diff --git a/pkg/spanconfig/spanconfiglimiter/noop.go b/pkg/spanconfig/spanconfiglimiter/noop.go new file mode 100644 index 000000000000..423d192fc2b0 --- /dev/null +++ b/pkg/spanconfig/spanconfiglimiter/noop.go @@ -0,0 +1,28 @@ +// Copyright 2022 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 spanconfiglimiter + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/spanconfig" +) + +var _ spanconfig.Limiter = &NoopLimiter{} + +// NoopLimiter is a Limiter that simply no-ops (i.e. doesn't limit anything). +type NoopLimiter struct{} + +// ShouldLimit is part of the spanconfig.Limiter interface. +func (n NoopLimiter) ShouldLimit(context.Context, *kv.Txn, int) (bool, error) { + return false, nil +} diff --git a/pkg/spanconfig/spanconfigsplitter/BUILD.bazel b/pkg/spanconfig/spanconfigsplitter/BUILD.bazel index 54127852ae4a..1473539fda39 100644 --- a/pkg/spanconfig/spanconfigsplitter/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsplitter/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "spanconfigsplitter", - srcs = ["splitter.go"], + srcs = [ + "noop.go", + "splitter.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsplitter", visibility = ["//visibility:public"], deps = [ diff --git a/pkg/spanconfig/spanconfigsplitter/noop.go b/pkg/spanconfig/spanconfigsplitter/noop.go new file mode 100644 index 000000000000..2cb74d538579 --- /dev/null +++ b/pkg/spanconfig/spanconfigsplitter/noop.go @@ -0,0 +1,28 @@ +// 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 spanconfigsplitter + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" +) + +var _ spanconfig.Splitter = &NoopSplitter{} + +// NoopSplitter is a Splitter that only returns "illegal use" errors. +type NoopSplitter struct{} + +// Splits is part of spanconfig.Splitter. +func (i NoopSplitter) Splits(context.Context, catalog.TableDescriptor) (int, error) { + return 0, nil +} diff --git a/pkg/spanconfig/spanconfigsplitter/splitter.go b/pkg/spanconfig/spanconfigsplitter/splitter.go index 27a253509d8a..97d58327d640 100644 --- a/pkg/spanconfig/spanconfigsplitter/splitter.go +++ b/pkg/spanconfig/spanconfigsplitter/splitter.go @@ -15,6 +15,7 @@ package spanconfigsplitter import ( "context" "fmt" + "reflect" "strings" "github.com/cockroachdb/cockroach/pkg/keys" @@ -106,6 +107,10 @@ func New(codec keys.SQLCodec, knobs *spanconfig.TestingKnobs) *Splitter { // descriptors that refer to them. This interface is used near by this GC // activity, so type information is not always available. func (s *Splitter) Splits(ctx context.Context, table catalog.TableDescriptor) (int, error) { + if isNil(table) { + return 0, nil // nothing to do + } + if s.knobs.ExcludeDroppedDescriptorsFromLookup && table.Dropped() { return 0, nil // we're excluding this descriptor; nothing to do here } @@ -197,3 +202,8 @@ type partition struct { index catalog.Index // index being partitioned level int // recursion level, used only for test-logging } + +func isNil(table catalog.TableDescriptor) bool { + vTable := reflect.ValueOf(table) + return vTable.Kind() == reflect.Ptr && vTable.IsNil() || table == nil +} diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go index 8ce7595eec53..d8eb4080d1c9 100644 --- a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go @@ -82,8 +82,15 @@ func (s *Tenant) updateTimestampAfterLastSQLChange() { // the execution timestamp for subsequent use. func (s *Tenant) Exec(query string, args ...interface{}) { s.db.Exec(s.t, query, args...) + s.updateTimestampAfterLastSQLChange() +} +// ExecWithErr is like Exec but returns the error, if any. It records the +// execution timestamp for subsequent use. +func (s *Tenant) ExecWithErr(query string, args ...interface{}) error { + _, err := s.db.DB.ExecContext(context.Background(), query, args...) s.updateTimestampAfterLastSQLChange() + return err } // TimestampAfterLastSQLChange returns a timestamp after the last time Exec was diff --git a/pkg/spanconfig/testing_knobs.go b/pkg/spanconfig/testing_knobs.go index 255d83b89e56..7474ba0a1c72 100644 --- a/pkg/spanconfig/testing_knobs.go +++ b/pkg/spanconfig/testing_knobs.go @@ -94,6 +94,10 @@ type TestingKnobs struct { // ProtectedTSReaderOverrideFn returns a ProtectedTSReader which is used to // override the ProtectedTSReader used when setting up a new store. ProtectedTSReaderOverrideFn func(clock *hlc.Clock) ProtectedTSReader + + // LimiterLimitOverride, if set, allows tests to dynamically override the span + // config limit. + LimiterLimitOverride func() int64 } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/sql/alter_column_type.go b/pkg/sql/alter_column_type.go index b1eec3e89293..57c590de651c 100644 --- a/pkg/sql/alter_column_type.go +++ b/pkg/sql/alter_column_type.go @@ -154,7 +154,7 @@ func AlterColumnType( if err := alterColumnTypeGeneral(ctx, tableDesc, col, typ, t.Using, params, cmds, tn); err != nil { return err } - if err := params.p.createOrUpdateSchemaChangeJob(params.ctx, tableDesc, tree.AsStringWithFQNames(t, params.Ann()), tableDesc.ClusterVersion.NextMutationID); err != nil { + if err := params.p.createOrUpdateSchemaChangeJob(params.ctx, tableDesc, tree.AsStringWithFQNames(t, params.Ann()), tableDesc.ClusterVersion().NextMutationID); err != nil { return err } params.p.BufferClientNotice(params.ctx, pgnotice.Newf("ALTER COLUMN TYPE changes are finalized asynchronously; "+ @@ -270,7 +270,7 @@ func alterColumnTypeGeneral( // Disallow ALTER COLUMN TYPE general if the table is already undergoing // a schema change. - currentMutationID := tableDesc.ClusterVersion.NextMutationID + currentMutationID := tableDesc.ClusterVersion().NextMutationID for i := range tableDesc.Mutations { mut := &tableDesc.Mutations[i] if mut.MutationID < currentMutationID { diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go index 39d8687eea1d..3311930787ae 100644 --- a/pkg/sql/alter_index.go +++ b/pkg/sql/alter_index.go @@ -154,7 +154,7 @@ func (n *alterIndexNode) startExec(params runParams) error { } mutationID := descpb.InvalidMutationID if addedMutations { - mutationID = n.tableDesc.ClusterVersion.NextMutationID + mutationID = n.tableDesc.ClusterVersion().NextMutationID } if err := params.p.writeSchemaChange( params.ctx, n.tableDesc, mutationID, tree.AsStringWithFQNames(n.n, params.Ann()), diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index c04e6b0b1907..795e0e1cc867 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -85,7 +85,7 @@ func (p *planner) AlterPrimaryKey( // Ensure that other schema changes on this table are not currently // executing, and that other schema changes have not been performed // in the current transaction. - currentMutationID := tableDesc.ClusterVersion.NextMutationID + currentMutationID := tableDesc.ClusterVersion().NextMutationID for i := range tableDesc.Mutations { mut := &tableDesc.Mutations[i] if mut.MutationID == currentMutationID { diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 017ba071d7aa..c61959acf208 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -847,7 +847,7 @@ func (n *alterTableNode) startExec(params runParams) error { mutationID := descpb.InvalidMutationID if addedMutations { - mutationID = n.tableDesc.ClusterVersion.NextMutationID + mutationID = n.tableDesc.ClusterVersion().NextMutationID } if err := params.p.writeSchemaChange( params.ctx, n.tableDesc, mutationID, tree.AsStringWithFQNames(n.n, params.Ann()), diff --git a/pkg/sql/alter_table_locality.go b/pkg/sql/alter_table_locality.go index 540bb1d0fc0f..d5bf46687e08 100644 --- a/pkg/sql/alter_table_locality.go +++ b/pkg/sql/alter_table_locality.go @@ -445,7 +445,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityFromOrToRegionalByRow( return params.p.writeSchemaChange( params.ctx, n.tableDesc, - n.tableDesc.ClusterVersion.NextMutationID, + n.tableDesc.ClusterVersion().NextMutationID, tree.AsStringWithFQNames(&n.n, params.Ann()), ) } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 7cc69587c6fb..542c7391c69e 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -2061,7 +2061,8 @@ func (sc *SchemaChanger) mergeFromTemporaryIndex( }); err != nil { return err } - tableDesc := tabledesc.NewBuilder(&tbl.ClusterVersion).BuildImmutableTable() + clusterVersion := tbl.ClusterVersion() + tableDesc := tabledesc.NewBuilder(&clusterVersion).BuildImmutableTable() if err := sc.distIndexMerge(ctx, tableDesc, addingIndexes, temporaryIndexes, fractionScaler); err != nil { return err } @@ -2425,7 +2426,7 @@ func validateCheckInTxn( checkExpr string, ) error { var syntheticDescs []catalog.Descriptor - if tableDesc.Version > tableDesc.ClusterVersion.Version { + if tableDesc.Version > tableDesc.ClusterVersion().Version { syntheticDescs = append(syntheticDescs, tableDesc) } ie := ief(ctx, sessionData) @@ -2456,7 +2457,7 @@ func validateFkInTxn( fkName string, ) error { var syntheticTable catalog.TableDescriptor - if srcTable.Version > srcTable.ClusterVersion.Version { + if srcTable.Version > srcTable.ClusterVersion().Version { syntheticTable = srcTable } var fk *descpb.ForeignKeyConstraint @@ -2507,7 +2508,7 @@ func validateUniqueWithoutIndexConstraintInTxn( constraintName string, ) error { var syntheticDescs []catalog.Descriptor - if tableDesc.Version > tableDesc.ClusterVersion.Version { + if tableDesc.Version > tableDesc.ClusterVersion().Version { syntheticDescs = append(syntheticDescs, tableDesc) } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 154b205cc657..4662378acb06 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -319,7 +319,9 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { target.AddDescriptorForSystemTenant(systemschema.SpanConfigurationsTable) // Tables introduced in 22.1. + target.AddDescriptorForSystemTenant(systemschema.TenantSettingsTable) + target.AddDescriptorForNonSystemTenant(systemschema.SpanCountTable) // Adding a new system table? It should be added here to the metadata schema, // and also created as a migration for older clusters. diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index aeceec0fbb88..3cbebb2e5dab 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -84,6 +84,7 @@ const ( SQLInstancesTableName SystemTableName = "sql_instances" SpanConfigurationsTableName SystemTableName = "span_configurations" TenantSettingsTableName SystemTableName = "tenant_settings" + SpanCountTableName SystemTableName = "span_count" ) // Oid for virtual database and table. diff --git a/pkg/sql/catalog/catprivilege/system.go b/pkg/sql/catalog/catprivilege/system.go index 08fb45b071aa..06e27d97ddf1 100644 --- a/pkg/sql/catalog/catprivilege/system.go +++ b/pkg/sql/catalog/catprivilege/system.go @@ -63,6 +63,7 @@ var ( catconstants.SQLInstancesTableName, catconstants.SpanConfigurationsTableName, catconstants.TenantSettingsTableName, + catconstants.SpanCountTableName, } systemSuperuserPrivileges = func() map[descpb.NameInfo]privilege.List { diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index d68b6be84499..5b7dc77b1c6a 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -36,6 +36,7 @@ go_library( "//pkg/kv", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/sql/catalog", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/sql/catalog/descs/factory.go b/pkg/sql/catalog/descs/factory.go index 89a608f0999a..038f15b7d2f6 100644 --- a/pkg/sql/catalog/descs/factory.go +++ b/pkg/sql/catalog/descs/factory.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" @@ -22,12 +23,14 @@ import ( // CollectionFactory is used to construct a new Collection. type CollectionFactory struct { - settings *cluster.Settings - codec keys.SQLCodec - leaseMgr *lease.Manager - virtualSchemas catalog.VirtualSchemas - hydratedTables *hydratedtables.Cache - systemDatabase *systemDatabaseNamespaceCache + settings *cluster.Settings + codec keys.SQLCodec + leaseMgr *lease.Manager + virtualSchemas catalog.VirtualSchemas + hydratedTables *hydratedtables.Cache + systemDatabase *systemDatabaseNamespaceCache + spanConfigSplitter spanconfig.Splitter + spanConfigLimiter spanconfig.Limiter } // NewCollectionFactory constructs a new CollectionFactory which holds onto @@ -37,14 +40,18 @@ func NewCollectionFactory( leaseMgr *lease.Manager, virtualSchemas catalog.VirtualSchemas, hydratedTables *hydratedtables.Cache, + spanConfigSplitter spanconfig.Splitter, + spanConfigLimiter spanconfig.Limiter, ) *CollectionFactory { return &CollectionFactory{ - settings: settings, - codec: leaseMgr.Codec(), - leaseMgr: leaseMgr, - virtualSchemas: virtualSchemas, - hydratedTables: hydratedTables, - systemDatabase: newSystemDatabaseNamespaceCache(leaseMgr.Codec()), + settings: settings, + codec: leaseMgr.Codec(), + leaseMgr: leaseMgr, + virtualSchemas: virtualSchemas, + hydratedTables: hydratedTables, + systemDatabase: newSystemDatabaseNamespaceCache(leaseMgr.Codec()), + spanConfigSplitter: spanConfigSplitter, + spanConfigLimiter: spanConfigLimiter, } } diff --git a/pkg/sql/catalog/descs/txn.go b/pkg/sql/catalog/descs/txn.go index dbe85df9fb5a..1d9e211a7327 100644 --- a/pkg/sql/catalog/descs/txn.go +++ b/pkg/sql/catalog/descs/txn.go @@ -17,8 +17,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/retry" @@ -90,6 +93,12 @@ func (cf *CollectionFactory) Txn( return err } modifiedDescriptors = descsCol.GetDescriptorsWithNewVersion() + + if err := CheckSpanCountLimit( + ctx, &descsCol, cf.spanConfigSplitter, cf.spanConfigLimiter, txn, + ); err != nil { + return err + } retryErr, err := CheckTwoVersionInvariant( ctx, db.Clock(), ie, &descsCol, txn, nil /* onRetryBackoff */) if retryErr { @@ -219,3 +228,44 @@ func CheckTwoVersionInvariant( } return true, retryErr } + +// CheckSpanCountLimit checks whether committing the set of uncommitted tables +// would exceed the span count limit we're allowed (applicable only to secondary +// tenants). +func CheckSpanCountLimit( + ctx context.Context, + descsCol *Collection, + splitter spanconfig.Splitter, + limiter spanconfig.Limiter, + txn *kv.Txn, +) error { + if !descsCol.codec().ForSystemTenant() { + var totalSpanCountDelta int + for _, ut := range descsCol.GetUncommittedTables() { + uncommittedMutTable, err := descsCol.GetUncommittedMutableTableByID(ut.GetID()) + if err != nil { + return err + } + + var originalTableDesc catalog.TableDescriptor + if originalDesc := uncommittedMutTable.OriginalDescriptor(); originalDesc != nil { + originalTableDesc = originalDesc.(catalog.TableDescriptor) + } + delta, err := spanconfig.Delta(ctx, splitter, originalTableDesc, uncommittedMutTable) + if err != nil { + return err + } + totalSpanCountDelta += delta + } + + shouldLimit, err := limiter.ShouldLimit(ctx, txn, totalSpanCountDelta) + if err != nil { + return err + } + if shouldLimit { + return pgerror.New(pgcode.ConfigurationLimitExceeded, "exceeded limit for number of table spans") + } + } + + return nil +} diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 389b3848d56c..bef93006ac86 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -667,6 +667,15 @@ CREATE TABLE system.tenant_settings ( CONSTRAINT "primary" PRIMARY KEY (tenant_id, name), FAMILY (tenant_id, name, value, last_updated, value_type, reason) );` + + SpanCountTableSchema = ` +CREATE TABLE system.span_count ( + singleton BOOL DEFAULT TRUE, + span_count INT NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (singleton), + CONSTRAINT single_row CHECK (singleton), + FAMILY "primary" (singleton, span_count) +);` ) func pk(name string) descpb.IndexDescriptor { @@ -2357,6 +2366,36 @@ var ( Version: descpb.StrictIndexColumnIDGuaranteesVersion, }, )) + + // SpanCountTable is the descriptor for the split count table. + SpanCountTable = registerSystemTable( + SpanCountTableSchema, + systemTable( + catconstants.SpanCountTableName, + descpb.InvalidID, // dynamically assigned + []descpb.ColumnDescriptor{ + {Name: "singleton", ID: 1, Type: types.Bool, DefaultExpr: &trueBoolString}, + {Name: "span_count", ID: 2, Type: types.Int}, + }, + []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + DefaultColumnID: 2, + ColumnNames: []string{"singleton", "span_count"}, + ColumnIDs: []descpb.ColumnID{1, 2}, + }, + }, + pk("singleton"), + ), + func(tbl *descpb.TableDescriptor) { + tbl.Checks = []*descpb.TableDescriptor_CheckConstraint{{ + Name: "single_row", + Expr: "singleton", + ColumnIDs: []descpb.ColumnID{1}, + }} + }, + ) ) type descRefByName struct { diff --git a/pkg/sql/catalog/tabledesc/safe_format_test.go b/pkg/sql/catalog/tabledesc/safe_format_test.go index b27e693083e5..739f8d3f43a9 100644 --- a/pkg/sql/catalog/tabledesc/safe_format_test.go +++ b/pkg/sql/catalog/tabledesc/safe_format_test.go @@ -240,7 +240,7 @@ func TestSafeMessage(t *testing.T) { mutable.Families[0].ColumnNames = append(mutable.Families[0].ColumnNames, "c") mutable.Families[0].ColumnIDs = append(mutable.Families[0].ColumnIDs, 5) mutable.ModificationTime = hlc.Timestamp{WallTime: 1e9} - mutable.ClusterVersion = *mutable.TableDesc() + mutable.TestingSetClusterVersion(*mutable.TableDesc()) return mutable.ImmutableCopy().(catalog.TableDescriptor) }, }, @@ -262,7 +262,7 @@ func TestSafeMessage(t *testing.T) { "Indexes: [{ID: 1, Unique: true, KeyColumns: [{ID: 1, Dir: ASC}]}]" + "}", f: func(mutable *tabledesc.Mutable) catalog.TableDescriptor { - mutable.ClusterVersion = *mutable.TableDesc() + mutable.TestingSetClusterVersion(*mutable.TableDesc()) return mutable.ImmutableCopy().(catalog.TableDescriptor) }, }, diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index dc21740b7076..f46b9215f590 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -48,8 +48,9 @@ import ( type Mutable struct { wrapper - // ClusterVersion represents the version of the table descriptor read from the store. - ClusterVersion descpb.TableDescriptor + // original represents the version of the table descriptor read from the + // store. + original *immutable } const ( @@ -983,7 +984,7 @@ func fitColumnToFamily(desc *Mutable, col descpb.ColumnDescriptor) (int, bool) { // MaybeIncrementVersion implements the MutableDescriptor interface. func (desc *Mutable) MaybeIncrementVersion() { // Already incremented, no-op. - if desc.Version == desc.ClusterVersion.Version+1 || desc.ClusterVersion.Version == 0 { + if desc.Version == desc.ClusterVersion().Version+1 || desc.ClusterVersion().Version == 0 { return } desc.Version++ @@ -996,17 +997,37 @@ func (desc *Mutable) MaybeIncrementVersion() { // OriginalName implements the MutableDescriptor interface. func (desc *Mutable) OriginalName() string { - return desc.ClusterVersion.Name + return desc.ClusterVersion().Name } // OriginalID implements the MutableDescriptor interface. func (desc *Mutable) OriginalID() descpb.ID { - return desc.ClusterVersion.ID + return desc.ClusterVersion().ID } // OriginalVersion implements the MutableDescriptor interface. func (desc *Mutable) OriginalVersion() descpb.DescriptorVersion { - return desc.ClusterVersion.Version + return desc.ClusterVersion().Version +} + +// ClusterVersion returns the version of the table descriptor read from the +// store, if any. +// +// TODO(ajwerner): Make this deal in catalog.TableDescriptor instead. +func (desc *Mutable) ClusterVersion() descpb.TableDescriptor { + if desc.original == nil { + return descpb.TableDescriptor{} + } + return desc.original.TableDescriptor +} + +// OriginalDescriptor returns the original state of the descriptor prior to +// the mutations. +func (desc *Mutable) OriginalDescriptor() catalog.Descriptor { + if desc.original != nil { + return desc.original + } + return nil } // FamilyHeuristicTargetBytes is the target total byte size of columns that the @@ -1263,7 +1284,7 @@ func (desc *Mutable) RenameColumnDescriptor(column catalog.Column, newColName st // It returns either an active column or a column that was added in the // same transaction that is currently running. func (desc *Mutable) FindActiveOrNewColumnByName(name tree.Name) (catalog.Column, error) { - currentMutationID := desc.ClusterVersion.NextMutationID + currentMutationID := desc.ClusterVersion().NextMutationID for _, col := range desc.DeletableColumns() { if col.ColName() == name && ((col.Public()) || @@ -2170,8 +2191,8 @@ func (desc *Mutable) addMutationWithNextID(m descpb.DescriptorMutation) { // For tables created in the same transaction the next mutation ID will // not have been allocated and the added mutation will use an invalid ID. // This is fine because the mutation will be processed immediately. - m.MutationID = desc.ClusterVersion.NextMutationID - desc.NextMutationID = desc.ClusterVersion.NextMutationID + 1 + m.MutationID = desc.ClusterVersion().NextMutationID + desc.NextMutationID = desc.ClusterVersion().NextMutationID + 1 desc.Mutations = append(desc.Mutations, m) } @@ -2233,7 +2254,7 @@ func (desc *wrapper) HasColumnBackfillMutation() bool { // IsNew returns true if the table was created in the current // transaction. func (desc *Mutable) IsNew() bool { - return desc.ClusterVersion.ID == descpb.InvalidID + return desc.ClusterVersion().ID == descpb.InvalidID } // ColumnsSelectors generates Select expressions for cols. diff --git a/pkg/sql/catalog/tabledesc/table_desc.go b/pkg/sql/catalog/tabledesc/table_desc.go index 3db457e49281..fb62cff41fd5 100644 --- a/pkg/sql/catalog/tabledesc/table_desc.go +++ b/pkg/sql/catalog/tabledesc/table_desc.go @@ -139,7 +139,8 @@ func (desc *Mutable) NewBuilder() catalog.DescriptorBuilder { // IsUncommittedVersion implements the Descriptor interface. func (desc *Mutable) IsUncommittedVersion() bool { - return desc.IsNew() || desc.GetVersion() != desc.ClusterVersion.GetVersion() + clusterVersion := desc.ClusterVersion() + return desc.IsNew() || desc.GetVersion() != clusterVersion.GetVersion() } // SetDrainingNames implements the MutableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index 5f507696bac7..de17bc019165 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -161,7 +161,7 @@ func (tdb *tableDescriptorBuilder) BuildExistingMutableTable() *Mutable { TableDescriptor: *tdb.maybeModified, changes: tdb.changes, }, - ClusterVersion: *tdb.original, + original: makeImmutable(tdb.original), } } diff --git a/pkg/sql/catalog/tabledesc/table_desc_test.go b/pkg/sql/catalog/tabledesc/table_desc_test.go index a5814bc3cbea..f34a27256144 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_test.go +++ b/pkg/sql/catalog/tabledesc/table_desc_test.go @@ -49,3 +49,9 @@ func TestMaybeIncrementVersion(t *testing.T) { require.Equal(t, descpb.DescriptorVersion(2), mut.GetVersion()) }) } + +// TestingSetClusterVersion is a test helper to override the original table +// descriptor. +func (desc *Mutable) TestingSetClusterVersion(d descpb.TableDescriptor) { + desc.original = makeImmutable(&d) +} diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 130139919cc5..4eba91d0524b 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -834,6 +834,17 @@ func (ex *connExecutor) checkDescriptorTwoVersionInvariant(ctx context.Context) if knobs := ex.server.cfg.SchemaChangerTestingKnobs; knobs != nil { inRetryBackoff = knobs.TwoVersionLeaseViolation } + + if err := descs.CheckSpanCountLimit( + ctx, + &ex.extraTxnState.descCollection, + ex.server.cfg.SpanConfigSplitter, + ex.server.cfg.SpanConfigLimiter, + ex.state.mu.txn, + ); err != nil { + return err + } + retryErr, err := descs.CheckTwoVersionInvariant( ctx, ex.server.cfg.Clock, diff --git a/pkg/sql/create_index.go b/pkg/sql/create_index.go index dc4ad595e328..35a4303178e9 100644 --- a/pkg/sql/create_index.go +++ b/pkg/sql/create_index.go @@ -689,7 +689,7 @@ func (n *createIndexNode) startExec(params runParams) error { index := n.tableDesc.Mutations[mutationIdx].GetIndex() indexName := index.Name - mutationID := n.tableDesc.ClusterVersion.NextMutationID + mutationID := n.tableDesc.ClusterVersion().NextMutationID if err := params.p.writeSchemaChange( params.ctx, n.tableDesc, mutationID, tree.AsStringWithFQNames(n.n, params.Ann()), ); err != nil { diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index f6eb454dfd2c..ef2a5e036a6a 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -522,7 +522,7 @@ func (p *planner) replaceViewDesc( // Compare toReplace against its ClusterVersion to verify if // its new set of columns is valid for a replacement view. if err := verifyReplacingViewColumns( - toReplace.ClusterVersion.Columns, + toReplace.ClusterVersion().Columns, toReplace.Columns, ); err != nil { return nil, err diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go index f12299ca492d..686069959218 100644 --- a/pkg/sql/drop_index.go +++ b/pkg/sql/drop_index.go @@ -264,7 +264,7 @@ func (n *dropIndexNode) finalizeDropColumn(params runParams, tableDesc *tabledes if err := tableDesc.AllocateIDs(params.ctx, version); err != nil { return err } - mutationID := tableDesc.ClusterVersion.NextMutationID + mutationID := tableDesc.ClusterVersion().NextMutationID if err := params.p.writeSchemaChange( params.ctx, tableDesc, mutationID, tree.AsStringWithFQNames(n.n, params.Ann()), ); err != nil { @@ -550,7 +550,7 @@ func (p *planner) dropIndexByName( return err } - mutationID := tableDesc.ClusterVersion.NextMutationID + mutationID := tableDesc.ClusterVersion().NextMutationID if err := p.writeSchemaChange(ctx, tableDesc, mutationID, jobDesc); err != nil { return err } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index ceff7c4015a5..7b064cb9d7db 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1289,6 +1289,13 @@ type ExecutorConfig struct { // and related migrations. SpanConfigReconciler spanconfig.Reconciler + // SpanConfigSplitter is used during migrations to seed system.span_count with + // the right number of tenant spans. + SpanConfigSplitter spanconfig.Splitter + + // SpanConfigLimiter is used to limit how many span configs installed. + SpanConfigLimiter spanconfig.Limiter + // SpanConfigKVAccessor is used when creating and deleting tenant // records. SpanConfigKVAccessor spanconfig.KVAccessor diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index c268e3e6e1a0..ba894222692d 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -71,6 +72,19 @@ func gcTables( return errors.Wrapf(err, "clearing data for table %d", table.GetID()) } + delta, err := spanconfig.Delta(ctx, execCfg.SpanConfigSplitter, table, nil /* uncommitted */) + if err != nil { + return err + } + + // Deduct from system.span_count appropriately. + if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + _, err := execCfg.SpanConfigLimiter.ShouldLimit(ctx, txn, delta) + return err + }); err != nil { + return errors.Wrapf(err, "deducting span count for table %d", table.GetID()) + } + // Finished deleting all the table data, now delete the table meta data. if err := sql.DeleteTableDescAndZoneConfig( ctx, execCfg.DB, execCfg.Settings, execCfg.Codec, table, diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 3df0eb998df0..c390850fafb7 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -76,6 +76,7 @@ public role_members table NULL 0 NULL public role_options table NULL 0 NULL public scheduled_jobs table NULL 0 NULL public settings table NULL 0 NULL +public span_count table NULL 0 NULL public sql_instances table NULL 0 NULL public sqlliveness table NULL 0 NULL public statement_bundle_chunks table NULL 0 NULL @@ -179,6 +180,7 @@ SELECT id FROM system.descriptor 43 44 46 +50 100 101 102 @@ -828,6 +830,16 @@ system public settings root GRANT true system public settings root INSERT true system public settings root SELECT true system public settings root UPDATE true +system public span_count admin DELETE true +system public span_count admin GRANT true +system public span_count admin INSERT true +system public span_count admin SELECT true +system public span_count admin UPDATE true +system public span_count root DELETE true +system public span_count root GRANT true +system public span_count root INSERT true +system public span_count root SELECT true +system public span_count root UPDATE true system public sql_instances admin DELETE true system public sql_instances admin GRANT true system public sql_instances admin INSERT true diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace index 295ba69ee910..24ff6f69b29d 100644 --- a/pkg/sql/logictest/testdata/logic_test/system_namespace +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -82,6 +82,7 @@ SELECT * FROM system.namespace 1 29 role_options 33 1 29 scheduled_jobs 37 1 29 settings 6 +1 29 span_count 50 1 29 sql_instances 46 1 29 sqlliveness 39 1 29 statement_bundle_chunks 34 diff --git a/pkg/sql/refresh_materialized_view.go b/pkg/sql/refresh_materialized_view.go index 296202dea1da..db46b896a4bc 100644 --- a/pkg/sql/refresh_materialized_view.go +++ b/pkg/sql/refresh_materialized_view.go @@ -118,7 +118,7 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error { return params.p.writeSchemaChange( params.ctx, n.desc, - n.desc.ClusterVersion.NextMutationID, + n.desc.ClusterVersion().NextMutationID, tree.AsStringWithFQNames(n.n, params.Ann()), ) } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index e319c6a65d7c..b5a7c5a91b7f 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -2792,10 +2792,10 @@ func (sc *SchemaChanger) queueCleanupJob( ctx context.Context, scDesc *tabledesc.Mutable, txn *kv.Txn, ) (jobspb.JobID, error) { // Create jobs for dropped columns / indexes to be deleted. - mutationID := scDesc.ClusterVersion.NextMutationID + mutationID := scDesc.ClusterVersion().NextMutationID span := scDesc.PrimaryIndexSpan(sc.execCfg.Codec) var spanList []jobspb.ResumeSpanList - for j := len(scDesc.ClusterVersion.Mutations); j < len(scDesc.Mutations); j++ { + for j := len(scDesc.ClusterVersion().Mutations); j < len(scDesc.Mutations); j++ { spanList = append(spanList, jobspb.ResumeSpanList{ ResumeSpans: roachpb.Spans{span}, diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 98db79fe961a..e886ce051417 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -134,7 +134,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( } } span := tableDesc.PrimaryIndexSpan(p.ExecCfg().Codec) - for i := len(tableDesc.ClusterVersion.Mutations) + len(spanList); i < len(tableDesc.Mutations); i++ { + for i := len(tableDesc.ClusterVersion().Mutations) + len(spanList); i < len(tableDesc.Mutations); i++ { var resumeSpans []roachpb.Span mut := tableDesc.Mutations[i] if mut.GetIndex() != nil && mut.GetIndex().UseDeletePreservingEncoding { diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 30836846c97b..91ab0d8e0453 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -177,7 +177,7 @@ func TestSystemTableLiterals(t *testing.T) { } } - const expectedNumberOfSystemTables = 38 + const expectedNumberOfSystemTables = 39 require.Equal(t, expectedNumberOfSystemTables, len(testcases)) for name, test := range testcases { diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 50723f24206a..0ffbdd9c4904 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -129,7 +129,7 @@ initial-keys tenant=system initial-keys tenant=5 ---- -73 keys: +75 keys: /Tenant/5/Table/3/1/1/2/1 /Tenant/5/Table/3/1/3/2/1 /Tenant/5/Table/3/1/4/2/1 @@ -165,6 +165,7 @@ initial-keys tenant=5 /Tenant/5/Table/3/1/43/2/1 /Tenant/5/Table/3/1/44/2/1 /Tenant/5/Table/3/1/46/2/1 + /Tenant/5/Table/3/1/50/2/1 /Tenant/5/Table/5/1/0/2/1 /Tenant/5/Table/7/1/0/0 /Tenant/5/NamespaceTable/30/1/0/0/"system"/4/1 @@ -191,6 +192,7 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"role_options"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"settings"/4/1 + /Tenant/5/NamespaceTable/30/1/1/29/"span_count"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"sql_instances"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"sqlliveness"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"statement_bundle_chunks"/4/1 @@ -208,7 +210,7 @@ initial-keys tenant=5 initial-keys tenant=999 ---- -73 keys: +75 keys: /Tenant/999/Table/3/1/1/2/1 /Tenant/999/Table/3/1/3/2/1 /Tenant/999/Table/3/1/4/2/1 @@ -244,6 +246,7 @@ initial-keys tenant=999 /Tenant/999/Table/3/1/43/2/1 /Tenant/999/Table/3/1/44/2/1 /Tenant/999/Table/3/1/46/2/1 + /Tenant/999/Table/3/1/50/2/1 /Tenant/999/Table/5/1/0/2/1 /Tenant/999/Table/7/1/0/0 /Tenant/999/NamespaceTable/30/1/0/0/"system"/4/1 @@ -270,6 +273,7 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"role_options"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"scheduled_jobs"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"settings"/4/1 + /Tenant/999/NamespaceTable/30/1/1/29/"span_count"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"sql_instances"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"sqlliveness"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"statement_bundle_chunks"/4/1 From 4f9c27b75f9f4ee04575b88a2c1c2dd414624ec8 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 5 Apr 2022 14:10:00 -0700 Subject: [PATCH 2/3] colexecproj: split up default cmp proj op file into two This commit splits up a single file containing two default comparison projection operators into two files. This is done in preparation of the following commit (which will move one of the operators to a different package). Release note: None --- Makefile | 3 +- pkg/gen/execgen.bzl | 3 +- pkg/sql/colexec/colexecproj/BUILD.bazel | 3 +- .../default_cmp_proj_const_op.eg.go | 78 +++++++++++++++++ ...oj_ops.eg.go => default_cmp_proj_op.eg.go} | 58 +------------ .../colexecproj/default_cmp_proj_ops_tmpl.go | 9 +- .../cmd/execgen/default_cmp_proj_ops_gen.go | 87 +++++++++++++++---- 7 files changed, 161 insertions(+), 80 deletions(-) create mode 100644 pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go rename pkg/sql/colexec/colexecproj/{default_cmp_proj_ops.eg.go => default_cmp_proj_op.eg.go} (60%) diff --git a/Makefile b/Makefile index c3113cf9833f..ddc7c2b208cf 100644 --- a/Makefile +++ b/Makefile @@ -865,7 +865,8 @@ EXECGEN_TARGETS = \ pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go \ pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go \ pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go \ - pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go \ + pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go \ + pkg/sql/colexec/colexecproj/default_cmp_proj_op.eg.go \ pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go \ pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go \ pkg/sql/colexec/colexecproj/proj_like_ops.eg.go \ diff --git a/pkg/gen/execgen.bzl b/pkg/gen/execgen.bzl index 88605fe96317..b64d321c6262 100644 --- a/pkg/gen/execgen.bzl +++ b/pkg/gen/execgen.bzl @@ -50,7 +50,8 @@ EXECGEN_SRCS = [ "//pkg/sql/colexec/colexecjoin:mergejoiner_rightanti.eg.go", "//pkg/sql/colexec/colexecjoin:mergejoiner_rightouter.eg.go", "//pkg/sql/colexec/colexecjoin:mergejoiner_rightsemi.eg.go", - "//pkg/sql/colexec/colexecproj:default_cmp_proj_ops.eg.go", + "//pkg/sql/colexec/colexecproj:default_cmp_proj_const_op.eg.go", + "//pkg/sql/colexec/colexecproj:default_cmp_proj_op.eg.go", "//pkg/sql/colexec/colexecproj:proj_const_left_ops.eg.go", "//pkg/sql/colexec/colexecproj:proj_const_right_ops.eg.go", "//pkg/sql/colexec/colexecproj:proj_like_ops.eg.go", diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 7a27207cc880..1b78ccb98b0a 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -76,7 +76,8 @@ go_test( # Map between target name and relevant template. targets = [ - ("default_cmp_proj_ops.eg.go", "default_cmp_proj_ops_tmpl.go"), + ("default_cmp_proj_const_op.eg.go", "default_cmp_proj_ops_tmpl.go"), + ("default_cmp_proj_op.eg.go", "default_cmp_proj_ops_tmpl.go"), ("proj_const_left_ops.eg.go", "proj_const_ops_tmpl.go"), ("proj_const_right_ops.eg.go", "proj_const_ops_tmpl.go"), ("proj_like_ops.eg.go", "proj_const_ops_tmpl.go"), diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go b/pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go new file mode 100644 index 000000000000..2e940db20cf1 --- /dev/null +++ b/pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go @@ -0,0 +1,78 @@ +// Code generated by execgen; DO NOT EDIT. + +// Copyright 2020 The Cockroach Authors. +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colexecproj + +import ( + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +type defaultCmpRConstProjOp struct { + projConstOpBase + constArg tree.Datum + + adapter colexeccmp.ComparisonExprAdapter + toDatumConverter *colconv.VecToDatumConverter + datumToVecConverter func(tree.Datum) interface{} +} + +var _ colexecop.Operator = &defaultCmpRConstProjOp{} +var _ execinfra.Releasable = &defaultCmpRConstProjOp{} + +func (d *defaultCmpRConstProjOp) Next() coldata.Batch { + batch := d.Input.Next() + n := batch.Length() + if n == 0 { + return coldata.ZeroBatch + } + sel := batch.Selection() + output := batch.ColVec(d.outputIdx) + d.allocator.PerformOperation([]coldata.Vec{output}, func() { + d.toDatumConverter.ConvertBatchAndDeselect(batch) + nonConstColumn := d.toDatumConverter.GetDatumColumn(d.colIdx) + _ = nonConstColumn[n-1] + if sel != nil { + _ = sel[n-1] + } + for i := 0; i < n; i++ { + // Note that we performed a conversion with deselection, so there + // is no need to check whether sel is non-nil. + //gcassert:bce + res, err := d.adapter.Eval(nonConstColumn[i], d.constArg) + if err != nil { + colexecerror.ExpectedError(err) + } + rowIdx := i + if sel != nil { + rowIdx = sel[i] + } + // Convert the datum into a physical type and write it out. + // TODO(yuzefovich): this code block is repeated in several places. + // Refactor it. + if res == tree.DNull { + output.Nulls().SetNull(rowIdx) + } else { + converted := d.datumToVecConverter(res) + coldata.SetValueAt(output, converted, rowIdx) + } + } + }) + return batch +} + +func (d *defaultCmpRConstProjOp) Release() { + d.toDatumConverter.Release() +} diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go b/pkg/sql/colexec/colexecproj/default_cmp_proj_op.eg.go similarity index 60% rename from pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go rename to pkg/sql/colexec/colexecproj/default_cmp_proj_op.eg.go index 57bdb00a6775..c80fea43de12 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/default_cmp_proj_op.eg.go @@ -1,4 +1,5 @@ // Code generated by execgen; DO NOT EDIT. + // Copyright 2020 The Cockroach Authors. // Use of this software is governed by the Business Source License // included in the file licenses/BSL.txt. @@ -76,60 +77,3 @@ func (d *defaultCmpProjOp) Next() coldata.Batch { func (d *defaultCmpProjOp) Release() { d.toDatumConverter.Release() } - -type defaultCmpRConstProjOp struct { - projConstOpBase - constArg tree.Datum - - adapter colexeccmp.ComparisonExprAdapter - toDatumConverter *colconv.VecToDatumConverter - datumToVecConverter func(tree.Datum) interface{} -} - -var _ colexecop.Operator = &defaultCmpRConstProjOp{} -var _ execinfra.Releasable = &defaultCmpRConstProjOp{} - -func (d *defaultCmpRConstProjOp) Next() coldata.Batch { - batch := d.Input.Next() - n := batch.Length() - if n == 0 { - return coldata.ZeroBatch - } - sel := batch.Selection() - output := batch.ColVec(d.outputIdx) - d.allocator.PerformOperation([]coldata.Vec{output}, func() { - d.toDatumConverter.ConvertBatchAndDeselect(batch) - nonConstColumn := d.toDatumConverter.GetDatumColumn(d.colIdx) - _ = nonConstColumn[n-1] - if sel != nil { - _ = sel[n-1] - } - for i := 0; i < n; i++ { - // Note that we performed a conversion with deselection, so there - // is no need to check whether sel is non-nil. - //gcassert:bce - res, err := d.adapter.Eval(nonConstColumn[i], d.constArg) - if err != nil { - colexecerror.ExpectedError(err) - } - rowIdx := i - if sel != nil { - rowIdx = sel[i] - } - // Convert the datum into a physical type and write it out. - // TODO(yuzefovich): this code block is repeated in several places. - // Refactor it. - if res == tree.DNull { - output.Nulls().SetNull(rowIdx) - } else { - converted := d.datumToVecConverter(res) - coldata.SetValueAt(output, converted, rowIdx) - } - } - }) - return batch -} - -func (d *defaultCmpRConstProjOp) Release() { - d.toDatumConverter.Release() -} diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go index b1d5c1127385..99f8e98426f1 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go @@ -13,9 +13,10 @@ // +build execgen_template // -// This file is the execgen template for default_cmp_proj_ops.eg.go. It's -// formatted in a special way, so it's both valid Go and a valid text/template -// input. This permits editing this file with editor support. +// This file is the execgen template for default_cmp_proj_op.eg.go and +// default_cmp_proj_const_op.eg.go. It's formatted in a special way, so it's +// both valid Go and a valid text/template input. This permits editing this file +// with editor support. // // */}} @@ -31,7 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) -// {{range .}} +// {{define "defaultCmpProjOp"}} type defaultCmp_KINDProjOp struct { // {{if .IsRightConst}} diff --git a/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go index 446fce8ba89c..68e780d6f177 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go @@ -16,27 +16,82 @@ import ( "text/template" ) +// defaultCmpProjTemplate is the common base for the template used to generate +// code for default comparison projection operators. It should be used as a +// format string with one %s argument that specifies the package that the +// generated code is placed in. +const defaultCmpProjTemplate = ` +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package {{.TargetPkg}} + +import ( + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/sql/colconv" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +{{template "defaultCmpProjOp" .}} +` + const defaultCmpProjOpsTmpl = "pkg/sql/colexec/colexecproj/default_cmp_proj_ops_tmpl.go" -func genDefaultCmpProjOps(inputFileContents string, wr io.Writer) error { - s := strings.ReplaceAll(inputFileContents, "_KIND", "{{.Kind}}") +type defaultCmpProjOpOverload struct { + TargetPkg string + // Comparison operators are always normalized so that the constant is on the + // right side, so we skip generating the code when the constant is on the + // left. + IsRightConst bool + Kind string +} + +func getDefaultCmpProjOps(overload defaultCmpProjOpOverload) generator { + return func(inputFileContents string, wr io.Writer) error { + s := strings.ReplaceAll(inputFileContents, "_KIND", "{{.Kind}}") - tmpl, err := template.New("default_cmp_proj_ops").Parse(s) - if err != nil { - return err + tmpl, err := template.New("default_cmp_proj_ops").Parse(s) + if err != nil { + return err + } + tmpl, err = tmpl.Parse(defaultCmpProjTemplate) + if err != nil { + return err + } + return tmpl.Execute(wr, overload) } - return tmpl.Execute(wr, []struct { - // Comparison operators are always normalized so that the constant is - // on the right side, so we skip generating the code when the constant - // is on the left. - IsRightConst bool - Kind string - }{ - {IsRightConst: false, Kind: ""}, - {IsRightConst: true, Kind: "RConst"}, - }) } func init() { - registerGenerator(genDefaultCmpProjOps, "default_cmp_proj_ops.eg.go", defaultCmpProjOpsTmpl) + registerGenerator( + getDefaultCmpProjOps( + defaultCmpProjOpOverload{ + TargetPkg: "colexecproj", + IsRightConst: true, + Kind: "RConst", + }), + "default_cmp_proj_const_op.eg.go", /* outputFile */ + defaultCmpProjOpsTmpl, + ) + registerGenerator( + getDefaultCmpProjOps( + defaultCmpProjOpOverload{ + TargetPkg: "colexecproj", + IsRightConst: false, + Kind: "", + }), + "default_cmp_proj_op.eg.go", /* outputFile */ + defaultCmpProjOpsTmpl, + ) } From 2862dee06020a2f9a67643944a97f48fde000b3d Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 5 Apr 2022 13:39:01 -0700 Subject: [PATCH 3/3] colexecproj: extract a new package for projection ops with const This commit extracts a new `colexecprojconst` package out of `colexecproj` that contains all projection operators with one constant argument. This will allow for faster build speeds since both packages tens of thousands lines of code. Special care had to be taken for default comparison operator because we need to generate two files in different packages based on a single template. I followed the precedent of `sort_partitioner.eg.go` which had to do the same. Release note: None --- Makefile | 8 +- pkg/BUILD.bazel | 2 + pkg/gen/execgen.bzl | 8 +- pkg/sql/colexec/COLEXEC.bzl | 16 ++++ pkg/sql/colexec/colbuilder/BUILD.bazel | 1 + pkg/sql/colexec/colbuilder/execplan.go | 7 +- pkg/sql/colexec/colexecagg/BUILD.bazel | 1 + pkg/sql/colexec/colexecbase/BUILD.bazel | 1 + pkg/sql/colexec/colexechash/BUILD.bazel | 1 + pkg/sql/colexec/colexecjoin/BUILD.bazel | 1 + pkg/sql/colexec/colexecproj/BUILD.bazel | 28 +++--- .../colexecproj/proj_non_const_ops.eg.go | 12 --- .../colexecproj/proj_non_const_ops_tmpl.go | 12 --- .../colexecproj/projection_ops_test.go | 64 ------------- pkg/sql/colexec/colexecprojconst/BUILD.bazel | 90 +++++++++++++++++++ .../default_cmp_proj_const_op.eg.go | 2 +- .../like_ops.go | 2 +- .../proj_const_left_ops.eg.go | 2 +- .../colexecprojconst/proj_const_ops_base.go | 25 ++++++ .../proj_const_ops_tmpl.go | 2 +- .../proj_const_right_ops.eg.go | 2 +- .../proj_like_ops.eg.go | 2 +- .../colexecprojconst/projection_ops_test.go | 86 ++++++++++++++++++ pkg/sql/colexec/colexecsel/BUILD.bazel | 1 + pkg/sql/colexec/colexecspan/BUILD.bazel | 1 + pkg/sql/colexec/colexecwindow/BUILD.bazel | 1 + .../cmd/execgen/default_cmp_proj_ops_gen.go | 2 +- .../execgen/cmd/execgen/like_ops_gen.go | 17 ++-- .../execgen/cmd/execgen/projection_ops_gen.go | 2 +- .../opt/exec/execbuilder/testdata/tpch_vec | 68 +++++++------- .../exec/execbuilder/testdata/vectorize_local | 4 +- .../execbuilder/testdata/vectorize_overloads | 36 ++++---- 32 files changed, 326 insertions(+), 181 deletions(-) create mode 100644 pkg/sql/colexec/colexecprojconst/BUILD.bazel rename pkg/sql/colexec/{colexecproj => colexecprojconst}/default_cmp_proj_const_op.eg.go (98%) rename pkg/sql/colexec/{colexecproj => colexecprojconst}/like_ops.go (99%) rename pkg/sql/colexec/{colexecproj => colexecprojconst}/proj_const_left_ops.eg.go (99%) create mode 100644 pkg/sql/colexec/colexecprojconst/proj_const_ops_base.go rename pkg/sql/colexec/{colexecproj => colexecprojconst}/proj_const_ops_tmpl.go (99%) rename pkg/sql/colexec/{colexecproj => colexecprojconst}/proj_const_right_ops.eg.go (99%) rename pkg/sql/colexec/{colexecproj => colexecprojconst}/proj_like_ops.eg.go (99%) create mode 100644 pkg/sql/colexec/colexecprojconst/projection_ops_test.go diff --git a/Makefile b/Makefile index ddc7c2b208cf..12b1aa99cf62 100644 --- a/Makefile +++ b/Makefile @@ -865,12 +865,12 @@ EXECGEN_TARGETS = \ pkg/sql/colexec/colexecjoin/mergejoiner_rightanti.eg.go \ pkg/sql/colexec/colexecjoin/mergejoiner_rightouter.eg.go \ pkg/sql/colexec/colexecjoin/mergejoiner_rightsemi.eg.go \ - pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go \ pkg/sql/colexec/colexecproj/default_cmp_proj_op.eg.go \ - pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go \ - pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go \ - pkg/sql/colexec/colexecproj/proj_like_ops.eg.go \ pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go \ + pkg/sql/colexec/colexecprojconst/default_cmp_proj_const_op.eg.go \ + pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go \ + pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go \ + pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go \ pkg/sql/colexec/colexecsel/default_cmp_sel_ops.eg.go \ pkg/sql/colexec/colexecsel/selection_ops.eg.go \ pkg/sql/colexec/colexecsel/sel_like_ops.eg.go \ diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index f3b3dd24e70c..58e54edda001 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -248,6 +248,8 @@ ALL_TESTS = [ "//pkg/sql/colexec/colexecjoin:colexecjoin_test", "//pkg/sql/colexec/colexecproj:colexecproj_disallowed_imports_test", "//pkg/sql/colexec/colexecproj:colexecproj_test", + "//pkg/sql/colexec/colexecprojconst:colexecprojconst_disallowed_imports_test", + "//pkg/sql/colexec/colexecprojconst:colexecprojconst_test", "//pkg/sql/colexec/colexecsel:colexecsel_disallowed_imports_test", "//pkg/sql/colexec/colexecsel:colexecsel_test", "//pkg/sql/colexec/colexecspan:colexecspan_disallowed_imports_test", diff --git a/pkg/gen/execgen.bzl b/pkg/gen/execgen.bzl index b64d321c6262..de6d55ff3923 100644 --- a/pkg/gen/execgen.bzl +++ b/pkg/gen/execgen.bzl @@ -50,12 +50,12 @@ EXECGEN_SRCS = [ "//pkg/sql/colexec/colexecjoin:mergejoiner_rightanti.eg.go", "//pkg/sql/colexec/colexecjoin:mergejoiner_rightouter.eg.go", "//pkg/sql/colexec/colexecjoin:mergejoiner_rightsemi.eg.go", - "//pkg/sql/colexec/colexecproj:default_cmp_proj_const_op.eg.go", "//pkg/sql/colexec/colexecproj:default_cmp_proj_op.eg.go", - "//pkg/sql/colexec/colexecproj:proj_const_left_ops.eg.go", - "//pkg/sql/colexec/colexecproj:proj_const_right_ops.eg.go", - "//pkg/sql/colexec/colexecproj:proj_like_ops.eg.go", "//pkg/sql/colexec/colexecproj:proj_non_const_ops.eg.go", + "//pkg/sql/colexec/colexecprojconst:default_cmp_proj_const_op.eg.go", + "//pkg/sql/colexec/colexecprojconst:proj_const_left_ops.eg.go", + "//pkg/sql/colexec/colexecprojconst:proj_const_right_ops.eg.go", + "//pkg/sql/colexec/colexecprojconst:proj_like_ops.eg.go", "//pkg/sql/colexec/colexecsel:default_cmp_sel_ops.eg.go", "//pkg/sql/colexec/colexecsel:sel_like_ops.eg.go", "//pkg/sql/colexec/colexecsel:selection_ops.eg.go", diff --git a/pkg/sql/colexec/COLEXEC.bzl b/pkg/sql/colexec/COLEXEC.bzl index c3beb3552142..3a2e74429fda 100644 --- a/pkg/sql/colexec/COLEXEC.bzl +++ b/pkg/sql/colexec/COLEXEC.bzl @@ -13,3 +13,19 @@ $(location :goimports) -w $@ tools = [":execgen", ":goimports"], visibility = [":__pkg__", "//pkg/gen:__pkg__"], ) + +# Generating the code for `default_cmp_proj_const_op.eg.go` requires special +# handling because the template lives in a different package. +def gen_default_cmp_proj_const_rule(name, target, visibility = ["//visibility:private"]): + native.genrule( + name = name, + srcs = ["//pkg/sql/colexec/colexecproj:default_cmp_proj_ops_tmpl.go"], + outs = [target], + cmd = """\ +export COCKROACH_INTERNAL_DISABLE_METAMORPHIC_TESTING=true +$(location :execgen) -template $(SRCS) -fmt=false pkg/sql/colexec/colexecprojconst/$@ > $@ +$(location :goimports) -w $@ +""", + tools = [":execgen", ":goimports"], + visibility = [":__pkg__", "//pkg/gen:__pkg__"], + ) diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index 77d7a172e4ba..87e62389062a 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/sql/colexec/colexecbase", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecutils", "//pkg/sql/colexec/colexecwindow", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 4ecd92664cdc..70b28f2535e6 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj" + "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecprojconst" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecutils" "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow" @@ -2335,7 +2336,7 @@ func planProjectionExpr( resultIdx = len(typs) // The projection result will be outputted to a new column which is // appended to the input batch. - op, err = colexecproj.GetProjectionLConstOperator( + op, err = colexecprojconst.GetProjectionLConstOperator( allocator, typs, left.ResolvedType(), outputType, projOp, input, rightIdx, lConstArg, resultIdx, evalCtx, binFn, cmpExpr, ) @@ -2381,7 +2382,7 @@ func planProjectionExpr( switch cmpProjOp.Symbol { case treecmp.Like, treecmp.NotLike: negate := cmpProjOp.Symbol == treecmp.NotLike - op, err = colexecproj.GetLikeProjectionOperator( + op, err = colexecprojconst.GetLikeProjectionOperator( allocator, evalCtx, input, leftIdx, resultIdx, string(tree.MustBeDString(rConstArg)), negate, ) @@ -2412,7 +2413,7 @@ func planProjectionExpr( if op == nil || err != nil { // op hasn't been created yet, so let's try the constructor for // all other projection operators. - op, err = colexecproj.GetProjectionRConstOperator( + op, err = colexecprojconst.GetProjectionRConstOperator( allocator, typs, right.ResolvedType(), outputType, projOp, input, leftIdx, rConstArg, resultIdx, evalCtx, binFn, cmpExpr, ) diff --git a/pkg/sql/colexec/colexecagg/BUILD.bazel b/pkg/sql/colexec/colexecagg/BUILD.bazel index 10ee2571d2e4..25003bfb6939 100644 --- a/pkg/sql/colexec/colexecagg/BUILD.bazel +++ b/pkg/sql/colexec/colexecagg/BUILD.bazel @@ -79,6 +79,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexecbase/BUILD.bazel b/pkg/sql/colexec/colexecbase/BUILD.bazel index 1139b32b27ec..bfeb76cd52b4 100644 --- a/pkg/sql/colexec/colexecbase/BUILD.bazel +++ b/pkg/sql/colexec/colexecbase/BUILD.bazel @@ -109,6 +109,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexechash/BUILD.bazel b/pkg/sql/colexec/colexechash/BUILD.bazel index 32702cb6ea4c..e8d8763e6770 100644 --- a/pkg/sql/colexec/colexechash/BUILD.bazel +++ b/pkg/sql/colexec/colexechash/BUILD.bazel @@ -80,6 +80,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexecbase", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexecjoin/BUILD.bazel b/pkg/sql/colexec/colexecjoin/BUILD.bazel index 1641e6073b1a..7423cf60c15a 100644 --- a/pkg/sql/colexec/colexecjoin/BUILD.bazel +++ b/pkg/sql/colexec/colexecjoin/BUILD.bazel @@ -105,6 +105,7 @@ disallowed_imports_test( "//pkg/sql/colexec", "//pkg/sql/colexec/colexecagg", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 1b78ccb98b0a..0123603888a1 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -5,10 +5,9 @@ load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecproj", srcs = [ - "like_ops.go", ":gen-exec", # keep ], - importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", + importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", # keep visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", # keep @@ -17,22 +16,22 @@ go_library( "//pkg/server/telemetry", # keep "//pkg/sql/colconv", # keep "//pkg/sql/colexec/colexecbase", # keep - "//pkg/sql/colexec/colexeccmp", - "//pkg/sql/colexec/colexecutils", + "//pkg/sql/colexec/colexeccmp", # keep + "//pkg/sql/colexec/colexecutils", # keep "//pkg/sql/colexec/execgen", # keep "//pkg/sql/colexecerror", # keep - "//pkg/sql/colexecop", - "//pkg/sql/colmem", + "//pkg/sql/colexecop", # keep + "//pkg/sql/colmem", # keep "//pkg/sql/execinfra", # keep - "//pkg/sql/sem/tree", + "//pkg/sql/sem/tree", # keep "//pkg/sql/sem/tree/treebin", # keep "//pkg/sql/sem/tree/treecmp", # keep "//pkg/sql/sqltelemetry", # keep - "//pkg/sql/types", + "//pkg/sql/types", # keep "//pkg/util/duration", # keep "//pkg/util/json", # keep "@com_github_cockroachdb_apd_v3//:apd", # keep - "@com_github_cockroachdb_errors//:errors", + "@com_github_cockroachdb_errors//:errors", # keep ], ) @@ -44,7 +43,7 @@ go_test( "main_test.go", "projection_ops_test.go", ], - embed = [":colexecproj"], + embed = [":colexecproj"], # keep tags = ["no-remote"], deps = [ "//pkg/col/coldata", @@ -74,13 +73,13 @@ go_test( ], ) +# Export the template because it is used by another target in colexecprojconst +# package. +exports_files(["default_cmp_proj_ops_tmpl.go"]) + # Map between target name and relevant template. targets = [ - ("default_cmp_proj_const_op.eg.go", "default_cmp_proj_ops_tmpl.go"), ("default_cmp_proj_op.eg.go", "default_cmp_proj_ops_tmpl.go"), - ("proj_const_left_ops.eg.go", "proj_const_ops_tmpl.go"), - ("proj_const_right_ops.eg.go", "proj_const_ops_tmpl.go"), - ("proj_like_ops.eg.go", "proj_const_ops_tmpl.go"), ("proj_non_const_ops.eg.go", "proj_non_const_ops_tmpl.go"), ] @@ -100,6 +99,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexecagg", "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go index 6eb9dfcff23f..86f8c4fd7837 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go @@ -46,18 +46,6 @@ var ( _ apd.Context ) -// projConstOpBase contains all of the fields for projections with a constant, -// except for the constant itself. -// NOTE: this struct should be declared in proj_const_ops_tmpl.go, but if we do -// so, it'll be redeclared because we execute that template twice. To go -// around the problem we specify it here. -type projConstOpBase struct { - colexecop.OneInputHelper - allocator *colmem.Allocator - colIdx int - outputIdx int -} - // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { colexecop.OneInputHelper diff --git a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go index ca8cef7ea17e..008b6afef1c2 100644 --- a/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecproj/proj_non_const_ops_tmpl.go @@ -76,18 +76,6 @@ func _ASSIGN(_, _, _, _, _, _ interface{}) { // */}} -// projConstOpBase contains all of the fields for projections with a constant, -// except for the constant itself. -// NOTE: this struct should be declared in proj_const_ops_tmpl.go, but if we do -// so, it'll be redeclared because we execute that template twice. To go -// around the problem we specify it here. -type projConstOpBase struct { - colexecop.OneInputHelper - allocator *colmem.Allocator - colIdx int - outputIdx int -} - // projOpBase contains all of the fields for non-constant projections. type projOpBase struct { colexecop.OneInputHelper diff --git a/pkg/sql/colexec/colexecproj/projection_ops_test.go b/pkg/sql/colexec/colexecproj/projection_ops_test.go index 8bab832e2dc4..ef1d90715e3f 100644 --- a/pkg/sql/colexec/colexecproj/projection_ops_test.go +++ b/pkg/sql/colexec/colexecproj/projection_ops_test.go @@ -102,70 +102,6 @@ func TestProjDivFloat64Float64Op(t *testing.T) { }) } -func TestGetProjectionConstOperator(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - binOp := treebin.MakeBinaryOperator(treebin.Mult) - var input colexecop.Operator - colIdx := 3 - inputTypes := make([]*types.T, colIdx+1) - inputTypes[colIdx] = types.Float - constVal := 31.37 - constArg := tree.NewDFloat(tree.DFloat(constVal)) - outputIdx := 5 - op, err := GetProjectionRConstOperator( - testAllocator, inputTypes, types.Float, types.Float, binOp, input, colIdx, - constArg, outputIdx, nil /* EvalCtx */, nil /* BinFn */, nil, /* cmpExpr */ - ) - if err != nil { - t.Error(err) - } - expected := &projMultFloat64Float64ConstOp{ - projConstOpBase: projConstOpBase{ - OneInputHelper: colexecop.MakeOneInputHelper(op.(*projMultFloat64Float64ConstOp).Input), - allocator: testAllocator, - colIdx: colIdx, - outputIdx: outputIdx, - }, - constArg: constVal, - } - if !reflect.DeepEqual(op, expected) { - t.Errorf("got %+v,\nexpected %+v", op, expected) - } -} - -func TestGetProjectionConstMixedTypeOperator(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - cmpOp := treecmp.MakeComparisonOperator(treecmp.GE) - var input colexecop.Operator - colIdx := 3 - inputTypes := make([]*types.T, colIdx+1) - inputTypes[colIdx] = types.Int - constVal := int16(31) - constArg := tree.NewDInt(tree.DInt(constVal)) - outputIdx := 5 - op, err := GetProjectionRConstOperator( - testAllocator, inputTypes, types.Int2, types.Int, cmpOp, input, colIdx, - constArg, outputIdx, nil /* EvalCtx */, nil /* BinFn */, nil, /* cmpExpr */ - ) - if err != nil { - t.Error(err) - } - expected := &projGEInt64Int16ConstOp{ - projConstOpBase: projConstOpBase{ - OneInputHelper: colexecop.MakeOneInputHelper(op.(*projGEInt64Int16ConstOp).Input), - allocator: testAllocator, - colIdx: colIdx, - outputIdx: outputIdx, - }, - constArg: constVal, - } - if !reflect.DeepEqual(op, expected) { - t.Errorf("got %+v,\nexpected %+v", op, expected) - } -} - // TestRandomComparisons runs comparisons against all scalar types with random // non-null data verifying that the result of Datum.Compare matches the result // of the exec projection. diff --git a/pkg/sql/colexec/colexecprojconst/BUILD.bazel b/pkg/sql/colexec/colexecprojconst/BUILD.bazel new file mode 100644 index 000000000000..fec818e7ff00 --- /dev/null +++ b/pkg/sql/colexec/colexecprojconst/BUILD.bazel @@ -0,0 +1,90 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/sql/colexec:COLEXEC.bzl", "gen_default_cmp_proj_const_rule") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") + +go_library( + name = "colexecprojconst", + srcs = [ + "like_ops.go", + "proj_const_ops_base.go", + ":gen-default-cmp-proj-const-op", # keep + ":gen-exec", # keep + ], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecprojconst", + visibility = ["//visibility:public"], + deps = [ + "//pkg/col/coldata", # keep + "//pkg/col/coldataext", # keep + "//pkg/col/typeconv", # keep + "//pkg/server/telemetry", # keep + "//pkg/sql/colconv", # keep + "//pkg/sql/colexec/colexecbase", # keep + "//pkg/sql/colexec/colexeccmp", + "//pkg/sql/colexec/colexecutils", + "//pkg/sql/colexec/execgen", # keep + "//pkg/sql/colexecerror", # keep + "//pkg/sql/colexecop", + "//pkg/sql/colmem", + "//pkg/sql/execinfra", # keep + "//pkg/sql/sem/tree", + "//pkg/sql/sem/tree/treebin", # keep + "//pkg/sql/sem/tree/treecmp", # keep + "//pkg/sql/sqltelemetry", # keep + "//pkg/sql/types", + "//pkg/util/duration", # keep + "//pkg/util/json", # keep + "@com_github_cockroachdb_apd_v3//:apd", # keep + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "colexecprojconst_test", + srcs = ["projection_ops_test.go"], + embed = [":colexecprojconst"], + deps = [ + "//pkg/sql/colexecop", + "//pkg/sql/sem/tree", + "//pkg/sql/sem/tree/treebin", + "//pkg/sql/sem/tree/treecmp", + "//pkg/sql/types", + "//pkg/util/leaktest", + "//pkg/util/log", + ], +) + +# Map between target name and relevant template. +targets = [ + ("proj_const_left_ops.eg.go", "proj_const_ops_tmpl.go"), + ("proj_const_right_ops.eg.go", "proj_const_ops_tmpl.go"), + ("proj_like_ops.eg.go", "proj_const_ops_tmpl.go"), +] + +# Define a file group for all the .eg.go targets. +eg_go_filegroup( + name = "gen-exec", + targets = targets, +) + +# Define gen rules for individual eg.go files. +gen_eg_go_rules(targets) + +# Special-case the gen rule for default_cmp_proj_const_op.eg.go. +gen_default_cmp_proj_const_rule( + name = "gen-default-cmp-proj-const-op", + target = "default_cmp_proj_const_op.eg.go", +) + +disallowed_imports_test( + "colexecprojconst", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go b/pkg/sql/colexec/colexecprojconst/default_cmp_proj_const_op.eg.go similarity index 98% rename from pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go rename to pkg/sql/colexec/colexecprojconst/default_cmp_proj_const_op.eg.go index 2e940db20cf1..9bd3adfa90bb 100644 --- a/pkg/sql/colexec/colexecproj/default_cmp_proj_const_op.eg.go +++ b/pkg/sql/colexec/colexecprojconst/default_cmp_proj_const_op.eg.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecproj +package colexecprojconst import ( "github.com/cockroachdb/cockroach/pkg/col/coldata" diff --git a/pkg/sql/colexec/colexecproj/like_ops.go b/pkg/sql/colexec/colexecprojconst/like_ops.go similarity index 99% rename from pkg/sql/colexec/colexecproj/like_ops.go rename to pkg/sql/colexec/colexecprojconst/like_ops.go index 398b323140bb..962d16cb9c3e 100644 --- a/pkg/sql/colexec/colexecproj/like_ops.go +++ b/pkg/sql/colexec/colexecprojconst/like_ops.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecproj +package colexecprojconst import ( "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexeccmp" diff --git a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go similarity index 99% rename from pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go rename to pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go index d2e8cc71c40f..2a0d62bd0d08 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_left_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_left_ops.eg.go @@ -7,7 +7,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecproj +package colexecprojconst import ( "math" diff --git a/pkg/sql/colexec/colexecprojconst/proj_const_ops_base.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_base.go new file mode 100644 index 000000000000..429eccde3ce3 --- /dev/null +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_base.go @@ -0,0 +1,25 @@ +// Copyright 2022 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 colexecprojconst + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" +) + +// projConstOpBase contains all of the fields for projections with a constant, +// except for the constant itself. +type projConstOpBase struct { + colexecop.OneInputHelper + allocator *colmem.Allocator + colIdx int + outputIdx int +} diff --git a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go similarity index 99% rename from pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go rename to pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go index 8719cdf04087..fe878c65bd98 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go @@ -19,7 +19,7 @@ // // */}} -package colexecproj +package colexecprojconst import ( "github.com/cockroachdb/apd/v3" diff --git a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go similarity index 99% rename from pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go rename to pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go index d4d1470bd6be..64c61155d775 100644 --- a/pkg/sql/colexec/colexecproj/proj_const_right_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_const_right_ops.eg.go @@ -7,7 +7,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecproj +package colexecprojconst import ( "bytes" diff --git a/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go similarity index 99% rename from pkg/sql/colexec/colexecproj/proj_like_ops.eg.go rename to pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go index ea54a054f456..5e8df3c9a798 100644 --- a/pkg/sql/colexec/colexecproj/proj_like_ops.eg.go +++ b/pkg/sql/colexec/colexecprojconst/proj_like_ops.eg.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package colexecproj +package colexecprojconst import ( "bytes" diff --git a/pkg/sql/colexec/colexecprojconst/projection_ops_test.go b/pkg/sql/colexec/colexecprojconst/projection_ops_test.go new file mode 100644 index 000000000000..28893b19b51a --- /dev/null +++ b/pkg/sql/colexec/colexecprojconst/projection_ops_test.go @@ -0,0 +1,86 @@ +// Copyright 2022 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 colexecprojconst + +import ( + "reflect" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/colexecop" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestGetProjectionConstOperator(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + binOp := treebin.MakeBinaryOperator(treebin.Mult) + var input colexecop.Operator + colIdx := 3 + inputTypes := make([]*types.T, colIdx+1) + inputTypes[colIdx] = types.Float + constVal := 31.37 + constArg := tree.NewDFloat(tree.DFloat(constVal)) + outputIdx := 5 + op, err := GetProjectionRConstOperator( + nil /* allocator */, inputTypes, types.Float, types.Float, binOp, input, colIdx, + constArg, outputIdx, nil /* EvalCtx */, nil /* BinFn */, nil, /* cmpExpr */ + ) + if err != nil { + t.Error(err) + } + expected := &projMultFloat64Float64ConstOp{ + projConstOpBase: projConstOpBase{ + OneInputHelper: colexecop.MakeOneInputHelper(op.(*projMultFloat64Float64ConstOp).Input), + colIdx: colIdx, + outputIdx: outputIdx, + }, + constArg: constVal, + } + if !reflect.DeepEqual(op, expected) { + t.Errorf("got %+v,\nexpected %+v", op, expected) + } +} + +func TestGetProjectionConstMixedTypeOperator(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + cmpOp := treecmp.MakeComparisonOperator(treecmp.GE) + var input colexecop.Operator + colIdx := 3 + inputTypes := make([]*types.T, colIdx+1) + inputTypes[colIdx] = types.Int + constVal := int16(31) + constArg := tree.NewDInt(tree.DInt(constVal)) + outputIdx := 5 + op, err := GetProjectionRConstOperator( + nil /* allocator */, inputTypes, types.Int2, types.Int, cmpOp, input, colIdx, + constArg, outputIdx, nil /* EvalCtx */, nil /* BinFn */, nil, /* cmpExpr */ + ) + if err != nil { + t.Error(err) + } + expected := &projGEInt64Int16ConstOp{ + projConstOpBase: projConstOpBase{ + OneInputHelper: colexecop.MakeOneInputHelper(op.(*projGEInt64Int16ConstOp).Input), + colIdx: colIdx, + outputIdx: outputIdx, + }, + constArg: constVal, + } + if !reflect.DeepEqual(op, expected) { + t.Errorf("got %+v,\nexpected %+v", op, expected) + } +} diff --git a/pkg/sql/colexec/colexecsel/BUILD.bazel b/pkg/sql/colexec/colexecsel/BUILD.bazel index e5fdc3d904b9..04d6fadc7947 100644 --- a/pkg/sql/colexec/colexecsel/BUILD.bazel +++ b/pkg/sql/colexec/colexecsel/BUILD.bazel @@ -86,6 +86,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecwindow", ], ) diff --git a/pkg/sql/colexec/colexecspan/BUILD.bazel b/pkg/sql/colexec/colexecspan/BUILD.bazel index c54a22734f4c..90e83f79a04a 100644 --- a/pkg/sql/colexec/colexecspan/BUILD.bazel +++ b/pkg/sql/colexec/colexecspan/BUILD.bazel @@ -94,6 +94,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", "//pkg/sql/colexec/colexecwindow", ], diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index 8ce456a0ac65..1bc0aebe9786 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -123,6 +123,7 @@ disallowed_imports_test( "//pkg/sql/colexec/colexechash", "//pkg/sql/colexec/colexecjoin", "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecprojconst", "//pkg/sql/colexec/colexecsel", ], ) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go index 68e780d6f177..f938869134be 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/default_cmp_proj_ops_gen.go @@ -77,7 +77,7 @@ func init() { registerGenerator( getDefaultCmpProjOps( defaultCmpProjOpOverload{ - TargetPkg: "colexecproj", + TargetPkg: "colexecprojconst", IsRightConst: true, Kind: "RConst", }), diff --git a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go index 9c39001b88c2..ffe2a5e0fa50 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/like_ops_gen.go @@ -24,8 +24,10 @@ import ( // are several different implementations which may be chosen depending on the // complexity of the LIKE pattern. // -// likeTemplate needs to be used as a format string expecting exactly one %s -// arguments that describes the type of the operator (either "sel" or "proj"). +// likeTemplate needs to be used as a format string expecting exactly two %s +// arguments: +// [1]: is the package suffix (either "sel" or "projconst") +// [2]: is the type of the operator (either "sel" or "proj"). const likeTemplate = ` // Copyright 2019 The Cockroach Authors. // @@ -48,12 +50,15 @@ import ( ) {{range .}} -{{template "%[1]sConstOp" .}} +{{template "%[2]sConstOp" .}} {{end}} ` func genLikeOps( tmplGetter func(inputFileContents string) (*template.Template, error), + // pkgSuffix is the suffix of the package for the operator to be generated + // in (either "sel" or "projconst"). + pkgSuffix string, // opType is the type of the operator to be generated (either "sel" or // "proj"). opType string, @@ -63,7 +68,7 @@ func genLikeOps( if err != nil { return err } - tmpl, err = tmpl.Parse(fmt.Sprintf(likeTemplate, opType)) + tmpl, err = tmpl.Parse(fmt.Sprintf(likeTemplate, pkgSuffix, opType)) if err != nil { return err } @@ -147,6 +152,6 @@ func init() { projTemplate := replaceProjConstTmplVariables(inputFileContents, false /* isConstLeft */) return template.New("proj_like_ops").Funcs(template.FuncMap{"buildDict": buildDict}).Parse(projTemplate) } - registerGenerator(genLikeOps(getProjectionOpsTmpl, "proj"), "proj_like_ops.eg.go", projConstOpsTmpl) - registerGenerator(genLikeOps(getSelectionOpsTmpl, "sel"), "sel_like_ops.eg.go", selectionOpsTmpl) + registerGenerator(genLikeOps(getProjectionOpsTmpl, "projconst", "proj"), "proj_like_ops.eg.go", projConstOpsTmpl) + registerGenerator(genLikeOps(getSelectionOpsTmpl, "sel", "sel"), "sel_like_ops.eg.go", selectionOpsTmpl) } diff --git a/pkg/sql/colexec/execgen/cmd/execgen/projection_ops_gen.go b/pkg/sql/colexec/execgen/cmd/execgen/projection_ops_gen.go index 6d4ef99aaf27..d9cbc4747717 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/projection_ops_gen.go +++ b/pkg/sql/colexec/execgen/cmd/execgen/projection_ops_gen.go @@ -16,7 +16,7 @@ import ( "text/template" ) -const projConstOpsTmpl = "pkg/sql/colexec/colexecproj/proj_const_ops_tmpl.go" +const projConstOpsTmpl = "pkg/sql/colexec/colexecprojconst/proj_const_ops_tmpl.go" // replaceProjTmplVariables replaces template variables used in the templates // for projection operators. It should only be used within this file. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec index ea49ace203ec..e921ea5295c4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec +++ b/pkg/sql/opt/exec/execbuilder/testdata/tpch_vec @@ -20536,11 +20536,11 @@ EXPLAIN (VEC) SELECT l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, sum └ *colexec.sortOp └ *colexec.hashAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projPlusFloat64Float64ConstOp + └ *colexecprojconst.projPlusFloat64Float64ConstOp └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexecsel.selLEInt64Int64ConstOp └ *colfetcher.ColBatchScan @@ -20607,7 +20607,7 @@ EXPLAIN (VEC) SELECT n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue └ *colexec.sortOp └ *colexec.hashAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexecjoin.hashJoiner ├ *rowexec.joinReader │ └ *colexecjoin.hashJoiner @@ -20643,7 +20643,7 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR └ *colexec.sortOp └ *colexec.hashAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexec.defaultBuiltinFuncOperator └ *colexecbase.constBytesOp └ *colexecjoin.hashJoiner @@ -20660,13 +20660,13 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR ├ *colexecbase.constBoolOp │ └ *colexec.andProjOp │ ├ *colexec.bufferOp - │ ├ *colexecproj.projEQBytesBytesConstOp - │ └ *colexecproj.projEQBytesBytesConstOp + │ ├ *colexecprojconst.projEQBytesBytesConstOp + │ └ *colexecprojconst.projEQBytesBytesConstOp ├ *colexecbase.constBoolOp │ └ *colexec.andProjOp │ ├ *colexec.bufferOp - │ ├ *colexecproj.projEQBytesBytesConstOp - │ └ *colexecproj.projEQBytesBytesConstOp + │ ├ *colexecprojconst.projEQBytesBytesConstOp + │ └ *colexecprojconst.projEQBytesBytesConstOp └ *colexecbase.constBoolOp └ *colexec.bufferOp @@ -20682,7 +20682,7 @@ EXPLAIN (VEC) SELECT o_year, sum(CASE WHEN nation = 'BRAZIL' THEN volume ELSE 0 └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexecproj.projMultFloat64Float64Op - │ └ *colexecproj.projMinusFloat64ConstFloat64Op + │ └ *colexecprojconst.projMinusFloat64ConstFloat64Op │ └ *colexec.defaultBuiltinFuncOperator │ └ *colexecbase.constBytesOp │ └ *colexecjoin.hashJoiner @@ -20699,7 +20699,7 @@ EXPLAIN (VEC) SELECT o_year, sum(CASE WHEN nation = 'BRAZIL' THEN volume ELSE 0 │ │ └ *colexecsel.selEQBytesBytesConstOp │ │ └ *colfetcher.ColBatchScan │ └ *colfetcher.ColBatchScan - ├ *colexecproj.projEQBytesBytesConstOp + ├ *colexecprojconst.projEQBytesBytesConstOp │ └ *colexec.bufferOp └ *colexecbase.constFloat64Op └ *colexec.bufferOp @@ -20715,7 +20715,7 @@ EXPLAIN (VEC) SELECT nation, o_year, sum(amount) AS sum_profit FROM ( SELECT n_n └ *colexecproj.projMinusFloat64Float64Op └ *colexecproj.projMultFloat64Float64Op └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexec.defaultBuiltinFuncOperator └ *colexecbase.constBytesOp └ *colexecjoin.hashJoiner @@ -20739,7 +20739,7 @@ EXPLAIN (VEC) SELECT c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) └ *colexec.topKSorter └ *colexec.hashAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexecjoin.hashJoiner ├ *colexecjoin.hashJoiner │ ├ *colfetcher.ColBatchScan @@ -20801,10 +20801,10 @@ EXPLAIN (VEC) SELECT 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extended │ └ Node 1 └ *colexecproj.projDivFloat64Float64Op - └ *colexecproj.projMultFloat64Float64ConstOp + └ *colexecprojconst.projMultFloat64Float64ConstOp └ *colexec.orderedAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexecjoin.hashJoiner @@ -20812,8 +20812,8 @@ EXPLAIN (VEC) SELECT 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extended │ └ *colfetcher.ColIndexJoin │ └ *colfetcher.ColBatchScan ├ *colexecproj.projMultFloat64Float64Op - │ └ *colexecproj.projMinusFloat64ConstFloat64Op - │ └ *colexecproj.projPrefixBytesBytesConstOp + │ └ *colexecprojconst.projMinusFloat64ConstFloat64Op + │ └ *colexecprojconst.projPrefixBytesBytesConstOp │ └ *colexec.bufferOp └ *colexecbase.constFloat64Op └ *colexec.bufferOp @@ -20835,7 +20835,7 @@ EXPLAIN (VEC) SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM s └ *colexecbase.constNullOp └ *colexec.hashAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colfetcher.ColIndexJoin └ *colfetcher.ColBatchScan @@ -20866,11 +20866,11 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, par ---- │ └ Node 1 - └ *colexecproj.projDivFloat64Float64ConstOp + └ *colexecprojconst.projDivFloat64Float64ConstOp └ *colexec.orderedAggregator └ *rowexec.joinReader └ *rowexec.joinReader - └ *colexecproj.projMultFloat64Float64ConstOp + └ *colexecprojconst.projMultFloat64Float64ConstOp └ *colexec.orderedAggregator └ *rowexec.joinReader └ *rowexec.joinReader @@ -20904,7 +20904,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice* (1 - l_discount)) AS revenue FROM line └ Node 1 └ *colexec.orderedAggregator └ *colexecproj.projMultFloat64Float64Op - └ *colexecproj.projMinusFloat64ConstFloat64Op + └ *colexecprojconst.projMinusFloat64ConstFloat64Op └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexecjoin.hashJoiner @@ -20920,31 +20920,31 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice* (1 - l_discount)) AS revenue FROM line │ │ ├ *colexec.andProjOp │ │ │ ├ *colexec.andProjOp │ │ │ │ ├ *colexec.andProjOp - │ │ │ │ │ ├ *colexecproj.projEQBytesBytesConstOp + │ │ │ │ │ ├ *colexecprojconst.projEQBytesBytesConstOp │ │ │ │ │ └ *colexec.projectInOpBytes - │ │ │ │ └ *colexecproj.projGEFloat64Float64ConstOp - │ │ │ └ *colexecproj.projLEFloat64Float64ConstOp - │ │ └ *colexecproj.projLEInt64Int64ConstOp + │ │ │ │ └ *colexecprojconst.projGEFloat64Float64ConstOp + │ │ │ └ *colexecprojconst.projLEFloat64Float64ConstOp + │ │ └ *colexecprojconst.projLEInt64Int64ConstOp │ └ *colexec.andProjOp │ ├ *colexec.andProjOp │ │ ├ *colexec.andProjOp │ │ │ ├ *colexec.andProjOp - │ │ │ │ ├ *colexecproj.projEQBytesBytesConstOp + │ │ │ │ ├ *colexecprojconst.projEQBytesBytesConstOp │ │ │ │ └ *colexec.projectInOpBytes - │ │ │ └ *colexecproj.projGEFloat64Float64ConstOp - │ │ └ *colexecproj.projLEFloat64Float64ConstOp - │ └ *colexecproj.projLEInt64Int64ConstOp + │ │ │ └ *colexecprojconst.projGEFloat64Float64ConstOp + │ │ └ *colexecprojconst.projLEFloat64Float64ConstOp + │ └ *colexecprojconst.projLEInt64Int64ConstOp ├ *colexecbase.constBoolOp │ └ *colexec.andProjOp │ ├ *colexec.bufferOp │ ├ *colexec.andProjOp │ │ ├ *colexec.andProjOp │ │ │ ├ *colexec.andProjOp - │ │ │ │ ├ *colexecproj.projEQBytesBytesConstOp + │ │ │ │ ├ *colexecprojconst.projEQBytesBytesConstOp │ │ │ │ └ *colexec.projectInOpBytes - │ │ │ └ *colexecproj.projGEFloat64Float64ConstOp - │ │ └ *colexecproj.projLEFloat64Float64ConstOp - │ └ *colexecproj.projLEInt64Int64ConstOp + │ │ │ └ *colexecprojconst.projGEFloat64Float64ConstOp + │ │ └ *colexecprojconst.projLEFloat64Float64ConstOp + │ └ *colexecprojconst.projLEInt64Int64ConstOp └ *colexecbase.constBoolOp └ *colexec.bufferOp @@ -20960,7 +20960,7 @@ EXPLAIN (VEC) SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN └ *colexec.unorderedDistinct └ *rowexec.joinReader └ *colexecsel.selGTInt64Float64Op - └ *colexecproj.projMultFloat64Float64ConstOp + └ *colexecprojconst.projMultFloat64Float64ConstOp └ *colexec.hashAggregator └ *colexecjoin.hashJoiner ├ *colfetcher.ColIndexJoin diff --git a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local index 0d4e922ade8a..e850d6d8e103 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local +++ b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_local @@ -281,7 +281,7 @@ EXPLAIN (VEC) SELECT stddev((t46404_1.c0 > ANY (0, 0))::INT) FROM t46404_0, t464 └ Node 1 └ *colexec.hashAggregator └ *colexecbase.castBoolIntOp - └ *colexecproj.defaultCmpRConstProjOp + └ *colexecprojconst.defaultCmpRConstProjOp └ *colexecjoin.crossJoiner ├ *colfetcher.ColBatchScan └ *colfetcher.ColBatchScan @@ -328,7 +328,7 @@ EXPLAIN (VEC) SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 └ *colexecbase.castInt2IntOp └ *colexecbase.castInt2IntOp └ *colexecsel.selEQInt64Int64Op - └ *colexecproj.projPlusInt64Int64ConstOp + └ *colexecprojconst.projPlusInt64Int64ConstOp └ *colexecproj.projPlusInt64Int64Op └ *colexecbase.castInt4IntOp └ *colexecbase.castInt4IntOp diff --git a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_overloads b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_overloads index a7104449b2e7..22b4b21eb8ca 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/vectorize_overloads +++ b/pkg/sql/opt/exec/execbuilder/testdata/vectorize_overloads @@ -169,7 +169,7 @@ EXPLAIN (VEC) SELECT _inet - 1 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projMinusDatumInt64ConstOp + └ *colexecprojconst.projMinusDatumInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -186,7 +186,7 @@ EXPLAIN (VEC) SELECT 2 + _inet FROM many_types ---- │ └ Node 1 - └ *colexecproj.projPlusDatumInt64ConstOp + └ *colexecprojconst.projPlusDatumInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -242,7 +242,7 @@ EXPLAIN (VEC) SELECT _int << 1 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projLShiftInt64Int64ConstOp + └ *colexecprojconst.projLShiftInt64Int64ConstOp └ *colfetcher.ColBatchScan query T @@ -250,7 +250,7 @@ EXPLAIN (VEC) SELECT _int >> 1 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projRShiftInt64Int64ConstOp + └ *colexecprojconst.projRShiftInt64Int64ConstOp └ *colfetcher.ColBatchScan query T @@ -258,7 +258,7 @@ EXPLAIN (VEC) SELECT _varbit << 4 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projLShiftDatumInt64ConstOp + └ *colexecprojconst.projLShiftDatumInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -293,7 +293,7 @@ EXPLAIN (VEC) SELECT _varbit >> 4 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projRShiftDatumInt64ConstOp + └ *colexecprojconst.projRShiftDatumInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -353,7 +353,7 @@ EXPLAIN (VEC) SELECT _json -> 2 FROM many_types ---- │ └ Node 1 - └ *colexecproj.projJSONFetchValJSONInt64ConstOp + └ *colexecprojconst.projJSONFetchValJSONInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -361,8 +361,8 @@ EXPLAIN (VEC) SELECT _json -> 2 -> 'a' FROM many_types ---- │ └ Node 1 - └ *colexecproj.projJSONFetchValJSONBytesConstOp - └ *colexecproj.projJSONFetchValJSONInt64ConstOp + └ *colexecprojconst.projJSONFetchValJSONBytesConstOp + └ *colexecprojconst.projJSONFetchValJSONInt64ConstOp └ *colfetcher.ColBatchScan query T @@ -370,7 +370,7 @@ EXPLAIN (VEC) SELECT _json #> _stringarray, _json #> '{a,b}' FROM many_types ---- │ └ Node 1 - └ *colexecproj.projJSONFetchValPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchValPathJSONDatumConstOp └ *colexecproj.projJSONFetchValPathJSONDatumOp └ *colfetcher.ColBatchScan @@ -379,9 +379,9 @@ EXPLAIN (VEC) SELECT _json #> _stringarray #> '{c}', _json #> '{a,b}' #> '{c}' F ---- │ └ Node 1 - └ *colexecproj.projJSONFetchValPathJSONDatumConstOp - └ *colexecproj.projJSONFetchValPathJSONDatumConstOp - └ *colexecproj.projJSONFetchValPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchValPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchValPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchValPathJSONDatumConstOp └ *colexecproj.projJSONFetchValPathJSONDatumOp └ *colfetcher.ColBatchScan @@ -390,7 +390,7 @@ EXPLAIN (VEC) SELECT _json #>> _stringarray, _json #>> '{a,b}' FROM many_types ---- │ └ Node 1 - └ *colexecproj.projJSONFetchTextPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchTextPathJSONDatumConstOp └ *colexecproj.projJSONFetchTextPathJSONDatumOp └ *colfetcher.ColBatchScan @@ -399,8 +399,8 @@ EXPLAIN (VEC) SELECT _json #> '{a,b}' #>> '{c}' FROM many_types ---- │ └ Node 1 - └ *colexecproj.projJSONFetchTextPathJSONDatumConstOp - └ *colexecproj.projJSONFetchValPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchTextPathJSONDatumConstOp + └ *colexecprojconst.projJSONFetchValPathJSONDatumConstOp └ *colfetcher.ColBatchScan # Make sure we fall back to row engine when we have a mixed-type expression @@ -420,7 +420,7 @@ EXPLAIN (VEC) SELECT '[2, "hi", {"b": ["bar", {"c": 4}]}]'::jsonb -> _int FROM m ---- │ └ Node 1 - └ *colexecproj.projJSONFetchValJSONConstInt64Op + └ *colexecprojconst.projJSONFetchValJSONConstInt64Op └ *colfetcher.ColBatchScan # Check that the comparison expressions with the constant on the left are @@ -431,7 +431,7 @@ EXPLAIN (VEC) SELECT B'11' <> _varbit FROM many_types ---- │ └ Node 1 - └ *colexecproj.projNEDatumDatumConstOp + └ *colexecprojconst.projNEDatumDatumConstOp └ *colfetcher.ColBatchScan query T