From 1b7f6027a32cd5d4def5c03bc3a60d33403d974f Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Thu, 16 Sep 2021 13:38:16 -0400 Subject: [PATCH] sql: allow atomic name swaps This commit introduces a new cluster version, keyed as AvoidDrainingNames, which allows atomic name swaps between two tables/databases/schemas/types. This means we're no longer populating the draining names of a descriptor, instead we're updating the namespace table in the same transaction. The price to pay for all of this is that there is a period of time following a name swap in which the name may refer to either of the two descriptors. Fixes #54562. Release note (sql change): It's now possible to swap names (for tables, etc.) in the same transaction. For example: CREATE TABLE foo(); BEGIN; ALTER TABLE foo RENAME TO bar; CREATE TABLE foo(); COMMIT; Previously, we'd be getting a "relation ... already exists" error. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- .../testdata/benchmark_expectations | 24 ++-- pkg/ccl/backupccl/restore_job.go | 19 ++- pkg/ccl/importccl/BUILD.bazel | 1 + pkg/ccl/importccl/import_job.go | 37 +++--- pkg/clusterversion/cockroach_versions.go | 7 ++ pkg/clusterversion/key_string.go | 5 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/alter_schema.go | 38 +++--- pkg/sql/alter_table_set_schema.go | 25 ++-- pkg/sql/alter_type.go | 26 ++-- pkg/sql/catalog/lease/lease_test.go | 5 +- pkg/sql/catalog/typedesc/type_desc.go | 5 + pkg/sql/database.go | 31 +++-- pkg/sql/drop_database.go | 11 +- pkg/sql/drop_schema.go | 48 +++++--- pkg/sql/drop_table.go | 17 +-- pkg/sql/drop_test.go | 5 - pkg/sql/drop_type.go | 18 +-- .../logictest/testdata/logic_test/alter_type | 5 - .../logictest/testdata/logic_test/database | 19 --- .../logictest/testdata/logic_test/drop_type | 11 -- .../testdata/logic_test/rename_database | 64 ---------- .../testdata/logic_test/rename_table | 32 ----- pkg/sql/logictest/testdata/logic_test/schema | 31 ----- .../logictest/testdata/logic_test/sequences | 54 +-------- .../logictest/testdata/logic_test/set_schema | 3 - pkg/sql/logictest/testdata/logic_test/table | 52 -------- pkg/sql/logictest/testdata/logic_test/views | 58 --------- pkg/sql/name_util.go | 65 ++++++++++ .../testdata/show_trace_nonmetamorphic | 6 +- pkg/sql/rename_table.go | 37 ++---- pkg/sql/rename_test.go | 111 ------------------ pkg/sql/reparent_database.go | 36 +++--- pkg/sql/schema_changer_test.go | 62 ---------- pkg/sql/temporary_schema.go | 14 ++- 37 files changed, 286 insertions(+), 701 deletions(-) create mode 100644 pkg/sql/name_util.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 7c24361ae39e..d8f093c2694b 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -168,4 +168,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.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-4 set the active cluster version in the format '.' +version version 21.2-6 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 8924c5c2b7f6..ee1c321fcd14 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -173,6 +173,6 @@ trace.jaeger.agentstringthe address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as :. If no port is specified, 6381 will be used. 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.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion21.2-4set the active cluster version in the format '.' +versionversion21.2-6set the active cluster version in the format '.' diff --git a/pkg/bench/rttanalysis/testdata/benchmark_expectations b/pkg/bench/rttanalysis/testdata/benchmark_expectations index ccbc3ef164a1..1972fa19638e 100644 --- a/pkg/bench/rttanalysis/testdata/benchmark_expectations +++ b/pkg/bench/rttanalysis/testdata/benchmark_expectations @@ -32,21 +32,21 @@ exp,benchmark 20,CreateRole/create_role_with_3_options 18,CreateRole/create_role_with_no_options 19,DropDatabase/drop_database_0_tables -26,DropDatabase/drop_database_1_table -33,DropDatabase/drop_database_2_tables -40,DropDatabase/drop_database_3_tables +27,DropDatabase/drop_database_1_table +35,DropDatabase/drop_database_2_tables +43,DropDatabase/drop_database_3_tables 24,DropRole/drop_1_role 31,DropRole/drop_2_roles 38,DropRole/drop_3_roles -17,DropSequence/drop_1_sequence -24,DropSequence/drop_2_sequences -31,DropSequence/drop_3_sequences -19,DropTable/drop_1_table -27,DropTable/drop_2_tables -35,DropTable/drop_3_tables -20,DropView/drop_1_view -28,DropView/drop_2_views -36,DropView/drop_3_views +18,DropSequence/drop_1_sequence +26,DropSequence/drop_2_sequences +34,DropSequence/drop_3_sequences +20,DropTable/drop_1_table +29,DropTable/drop_2_tables +38,DropTable/drop_3_tables +21,DropView/drop_1_view +30,DropView/drop_2_views +39,DropView/drop_3_views 16,Grant/grant_all_on_1_table 18,Grant/grant_all_on_2_tables 20,Grant/grant_all_on_3_tables diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 9dd00aa5db41..225dda629770 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -2109,20 +2109,19 @@ func (r *restoreResumer) OnFailOrCancel(ctx context.Context, execCtx interface{} } } - if err := r.dropDescriptors(ctx, execCfg.JobRegistry, execCfg.Codec, txn, descsCol); err != nil { - return err - } - - if details.DescriptorCoverage == tree.AllDescriptors { - // The temporary system table descriptors should already have been dropped - // in `dropDescriptors` but we still need to drop the temporary system db. - return r.cleanupTempSystemTables(ctx, txn) - } - return nil + return r.dropDescriptors(ctx, execCfg.JobRegistry, execCfg.Codec, txn, descsCol) }); err != nil { return err } + if details.DescriptorCoverage == tree.AllDescriptors { + // The temporary system table descriptors should already have been dropped + // in `dropDescriptors` but we still need to drop the temporary system db. + if err := execCfg.DB.Txn(ctx, r.cleanupTempSystemTables); err != nil { + return err + } + } + // Emit to the event log that the job has completed reverting. emitRestoreJobEvent(ctx, p, jobs.StatusFailed, r.job) return nil diff --git a/pkg/ccl/importccl/BUILD.bazel b/pkg/ccl/importccl/BUILD.bazel index 23fa509f66f0..57e5da4a1bf2 100644 --- a/pkg/ccl/importccl/BUILD.bazel +++ b/pkg/ccl/importccl/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/ccl/storageccl", "//pkg/ccl/utilccl", "//pkg/cloud", + "//pkg/clusterversion", "//pkg/col/coldata", "//pkg/featureflag", "//pkg/jobs", diff --git a/pkg/ccl/importccl/import_job.go b/pkg/ccl/importccl/import_job.go index 1c43bdab10f1..c087d0412f63 100644 --- a/pkg/ccl/importccl/import_job.go +++ b/pkg/ccl/importccl/import_job.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -1485,25 +1486,33 @@ func (r *importResumer) dropSchemas( return nil, errors.Newf("unable to resolve schema desc with ID %d", schema.Desc.ID) } - //lint:ignore SA1019 deprecated method call is OK - schemaDesc.AddDrainingName(descpb.NameInfo{ - ParentID: details.ParentID, - ParentSchemaID: keys.RootNamespaceID, - Name: schemaDesc.Name, - }) - - // Update the parent database with information about the dropped schema. - if dbDesc.Schemas == nil { - dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) - } - dbDesc.Schemas[schema.Desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ID: dbDesc.ID, - Dropped: true} - // Mark the descriptor as dropped and write it to the batch. + // Delete namespace entry or update draining names depending on version. + schemaDesc.SetDropped() droppedSchemaIDs = append(droppedSchemaIDs, schemaDesc.GetID()) b := txn.NewBatch() + // TODO(postamar): remove version gate and else-block in 22.2 + if execCfg.Settings.Version.IsActive(ctx, clusterversion.AvoidDrainingNames) { + if dbDesc.Schemas != nil { + delete(dbDesc.Schemas, schemaDesc.GetName()) + } + b.Del(catalogkeys.EncodeNameKey(p.ExecCfg().Codec, schemaDesc)) + } else { + //lint:ignore SA1019 removal of deprecated method call scheduled for 22.2 + schemaDesc.AddDrainingName(descpb.NameInfo{ + ParentID: details.ParentID, + ParentSchemaID: keys.RootNamespaceID, + Name: schemaDesc.Name, + }) + // Update the parent database with information about the dropped schema. + if dbDesc.Schemas == nil { + dbDesc.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) + } + dbDesc.Schemas[schema.Desc.Name] = descpb.DatabaseDescriptor_SchemaInfo{ID: dbDesc.ID, Dropped: true} + } + if err := descsCol.WriteDescToBatch(ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), schemaDesc, b); err != nil { return nil, err diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 543f40892287..7b334de2fb71 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -276,6 +276,9 @@ const ( // requires the limit to always be overshot in order to properly enforce // limits when splitting requests. TargetBytesAvoidExcess + // AvoidDrainingNames avoids using the draining_names field when renaming or + // dropping descriptors. + AvoidDrainingNames // ************************************************* // Step (1): Add new versions here. @@ -473,6 +476,10 @@ var versionsSingleton = keyedVersions{ Key: TargetBytesAvoidExcess, Version: roachpb.Version{Major: 21, Minor: 2, Internal: 4}, }, + { + Key: AvoidDrainingNames, + Version: roachpb.Version{Major: 21, Minor: 2, Internal: 6}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 1d2e73449f2f..bf8670da4bda 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -47,11 +47,12 @@ func _() { _ = x[V21_2-36] _ = x[Start22_1-37] _ = x[TargetBytesAvoidExcess-38] + _ = x[AvoidDrainingNames-39] } -const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcess" +const _Key_name = "V21_1Start21_1PLUSStart21_2JoinTokensTableAcquisitionTypeInLeaseHistorySerializeViewUDTsExpressionIndexesDeleteDeprecatedNamespaceTableDescriptorMigrationFixDescriptorsDatabaseRoleSettingsTenantUsageTableSQLInstancesTableNewRetryableRangefeedErrorsAlterSystemWebSessionsCreateIndexesSeparatedIntentsMigrationPostSeparatedIntentsMigrationRetryJobsWithExponentialBackoffRecordsBasedRegistryAutoSpanConfigReconciliationJobPreventNewInterleavedTablesEnsureNoInterleavedTablesDefaultPrivilegesZonesTableForSecondaryTenantsUseKeyEncodeForHashShardedIndexesDatabasePlacementPolicyGeneratedAsIdentityOnUpdateExpressionsSpanConfigurationsTableBoundedStalenessDateAndIntervalStylePebbleFormatVersionedMarkerDataKeysRegistryPebbleSetWithDeleteTenantUsageSingleConsumptionColumnSQLStatsTablesSQLStatsCompactionScheduledJobV21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNames" -var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 446, 471, 488, 517, 550, 573, 592, 611, 634, 650, 670, 691, 713, 732, 766, 780, 810, 815, 824, 846} +var _Key_index = [...]uint16{0, 5, 18, 27, 42, 71, 88, 105, 154, 168, 188, 204, 221, 248, 283, 308, 337, 368, 388, 419, 446, 471, 488, 517, 550, 573, 592, 611, 634, 650, 670, 691, 713, 732, 766, 780, 810, 815, 824, 846, 864} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index baf7330b0662..6242a885d2a1 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -123,6 +123,7 @@ go_library( "lookup_join.go", "max_one_row.go", "mem_metrics.go", + "name_util.go", "notice.go", "opaque.go", "opt_catalog.go", diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go index 3774c3014bfe..8871b80625b6 100644 --- a/pkg/sql/alter_schema.go +++ b/pkg/sql/alter_schema.go @@ -14,10 +14,9 @@ import ( "context" "fmt" - "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" @@ -27,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" ) @@ -194,6 +192,12 @@ func (p *planner) setNewSchemaOwner( func (p *planner) renameSchema( ctx context.Context, db *dbdesc.Mutable, desc *schemadesc.Mutable, newName string, jobDesc string, ) error { + oldNameKey := descpb.NameInfo{ + ParentID: desc.GetParentID(), + ParentSchemaID: desc.GetParentSchemaID(), + Name: desc.GetName(), + } + // Check that there isn't a name collision with the new name. found, _, err := schemaExists(ctx, p.txn, p.ExecCfg().Codec, db.ID, newName) if err != nil { @@ -209,21 +213,12 @@ func (p *planner) renameSchema( } // Set the new name for the descriptor. - oldName := desc.Name - desc.AddDrainingName(descpb.NameInfo{ - ParentID: desc.ParentID, - ParentSchemaID: keys.RootNamespaceID, - Name: desc.Name, - }) + oldName := oldNameKey.GetName() desc.SetName(newName) - // Write a new namespace entry for the new name. - nameKey := catalogkeys.MakeSchemaNameKey(p.execCfg.Codec, desc.ParentID, newName) + // Write the new name and remove the old name. b := p.txn.NewBatch() - if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "CPut %s -> %d", nameKey, desc.ID) - } - b.CPut(nameKey, desc.ID, nil) + p.renameNamespaceEntry(ctx, b, oldNameKey, desc) if err := p.txn.Run(ctx, b); err != nil { return err } @@ -247,11 +242,16 @@ func (p *planner) renameSchema( ) } - // Mark the old schema name as dropped. - db.Schemas[oldName] = descpb.DatabaseDescriptor_SchemaInfo{ - ID: desc.ID, - Dropped: true, + // Remove the old schema name or mark it as dropped, depending on version. + if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.AvoidDrainingNames) { + delete(db.Schemas, oldName) + } else { + db.Schemas[oldName] = descpb.DatabaseDescriptor_SchemaInfo{ + ID: desc.ID, + Dropped: true, + } } + // Create an entry for the new schema name. db.Schemas[newName] = descpb.DatabaseDescriptor_SchemaInfo{ID: desc.ID} if err := p.writeNonDropDatabaseChange( diff --git a/pkg/sql/alter_table_set_schema.go b/pkg/sql/alter_table_set_schema.go index 1c3e558e8033..a7d7ae00150f 100644 --- a/pkg/sql/alter_table_set_schema.go +++ b/pkg/sql/alter_table_set_schema.go @@ -101,8 +101,11 @@ func (n *alterTableSetSchemaNode) startExec(params runParams) error { ctx := params.ctx p := params.p tableDesc := n.tableDesc - schemaID := tableDesc.GetParentSchemaID() - databaseID := tableDesc.GetParentID() + oldNameKey := descpb.NameInfo{ + ParentID: tableDesc.GetParentID(), + ParentSchemaID: tableDesc.GetParentSchemaID(), + Name: tableDesc.GetName(), + } kind := tree.GetTableType(tableDesc.IsSequence(), tableDesc.IsView(), tableDesc.GetIsMaterializedView()) oldName := tree.MakeTableNameFromPrefix(n.prefix.NamePrefix(), tree.Name(n.tableDesc.GetName())) @@ -114,38 +117,34 @@ func (n *alterTableSetSchemaNode) startExec(params runParams) error { // If the schema being changed to is the same as the current schema for the // table, do a no-op. - if desiredSchemaID == schemaID { + if desiredSchemaID == oldNameKey.GetParentSchemaID() { return nil } // TODO(ajwerner): Use the collection here. exists, _, err := catalogkv.LookupObjectID( - ctx, p.txn, p.ExecCfg().Codec, databaseID, desiredSchemaID, tableDesc.Name, + ctx, p.txn, p.ExecCfg().Codec, tableDesc.GetParentID(), desiredSchemaID, tableDesc.GetName(), ) if err == nil && exists { return pgerror.Newf(pgcode.DuplicateRelation, - "relation %s already exists in schema %s", tableDesc.Name, n.newSchema) + "relation %s already exists in schema %s", tableDesc.GetName(), n.newSchema) } else if err != nil { return err } - renameDetails := descpb.NameInfo{ - ParentID: databaseID, - ParentSchemaID: schemaID, - Name: tableDesc.Name, - } - tableDesc.AddDrainingName(renameDetails) - // Set the tableDesc's new schema id to the desired schema's id. tableDesc.SetParentSchemaID(desiredSchemaID) + b := p.txn.NewBatch() + p.renameNamespaceEntry(ctx, b, oldNameKey, tableDesc) + if err := p.writeSchemaChange( ctx, tableDesc, descpb.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann()), ); err != nil { return err } - if err := p.writeNameKey(ctx, tableDesc, tableDesc.ID); err != nil { + if err := p.txn.Run(ctx, b); err != nil { return err } diff --git a/pkg/sql/alter_type.go b/pkg/sql/alter_type.go index f497a35b2761..c67a4b03320a 100644 --- a/pkg/sql/alter_type.go +++ b/pkg/sql/alter_type.go @@ -275,23 +275,27 @@ func (p *planner) performRenameTypeDesc( newSchemaID descpb.ID, jobDesc string, ) error { - // Record the rename details in the descriptor for draining. - name := descpb.NameInfo{ - ParentID: desc.ParentID, - ParentSchemaID: desc.ParentSchemaID, - Name: desc.Name, + oldNameKey := descpb.NameInfo{ + ParentID: desc.GetParentID(), + ParentSchemaID: desc.GetParentSchemaID(), + Name: desc.GetName(), } - desc.AddDrainingName(name) - // Set the descriptor up with the new name. - desc.Name = newName - // Set the descriptor to the new schema ID. + // Update the type descriptor with the new name and new schema ID. + desc.SetName(newName) desc.SetParentSchemaID(newSchemaID) + + // Populate the namespace update batch. + b := p.txn.NewBatch() + p.renameNamespaceEntry(ctx, b, oldNameKey, desc) + + // Write the updated type descriptor. if err := p.writeTypeSchemaChange(ctx, desc, jobDesc); err != nil { return err } - // Write the new namespace key. - return p.writeNameKey(ctx, desc, desc.ID) + + // Run the namespace update batch. + return p.txn.Run(ctx, b) } func (p *planner) renameTypeValue( diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index 1116755621e3..3f6a713a90fe 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -563,6 +563,8 @@ CREATE TABLE test.t(a INT PRIMARY KEY); t.Fatal(err) } + tableDesc := catalogkv.TestingGetTableDescriptor(t.kvDB, keys.SystemSQLCodec, "test", "t") + // Block schema changers so that the table we're about to DROP is not actually // dropped; it will be left in a "deleted" state. mu.Lock() @@ -576,10 +578,9 @@ CREATE TABLE test.t(a INT PRIMARY KEY); } // Make sure we can't get a lease on the descriptor. - tableDesc := catalogkv.TestingGetTableDescriptor(t.kvDB, keys.SystemSQLCodec, "test", "t") // try to acquire at a bogus version to make sure we don't get back a lease we // already had. - _, err = t.acquireMinVersion(1, tableDesc.GetID(), tableDesc.GetVersion()+1) + _, err = t.acquireMinVersion(1, tableDesc.GetID(), tableDesc.GetVersion()+123) if !testutils.IsError(err, "descriptor is being dropped") { t.Fatalf("got a different error than expected: %v", err) } diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 665fb40d0ee0..378cd1b3e73b 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -451,6 +451,11 @@ func (desc *Mutable) AddDrainingName(name descpb.NameInfo) { desc.DrainingNames = append(desc.DrainingNames, name) } +// SetName sets the TypeDescriptor's name. +func (desc *Mutable) SetName(name string) { + desc.Name = name +} + // EnumMembers is a sortable list of TypeDescriptor_EnumMember, sorted by the // physical representation. type EnumMembers []descpb.TypeDescriptor_EnumMember diff --git a/pkg/sql/database.go b/pkg/sql/database.go index 02615b524d7e..166ab3382b96 100644 --- a/pkg/sql/database.go +++ b/pkg/sql/database.go @@ -13,17 +13,14 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/util/log" ) // @@ -38,9 +35,13 @@ import ( func (p *planner) renameDatabase( ctx context.Context, desc *dbdesc.Mutable, newName string, stmt string, ) error { - oldName := desc.GetName() - desc.SetName(newName) + oldNameKey := descpb.NameInfo{ + ParentID: desc.GetParentID(), + ParentSchemaID: desc.GetParentSchemaID(), + Name: desc.GetName(), + } + // Check that the new name is available. if exists, _, err := catalogkv.LookupDatabaseID(ctx, p.txn, p.ExecCfg().Codec, newName); err == nil && exists { return pgerror.Newf(pgcode.DuplicateDatabase, "the new database name %q already exists", newName) @@ -48,23 +49,19 @@ func (p *planner) renameDatabase( return err } - b := &kv.Batch{} - newKey := catalogkeys.MakeDatabaseNameKey(p.ExecCfg().Codec, newName) - descID := desc.GetID() - if p.ExtendedEvalContext().Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "CPut %s -> %d", newKey, descID) - } - b.CPut(newKey, descID, nil) + // Update the descriptor with the new name. + desc.SetName(newName) + + // Populate the namespace update batch. + b := p.txn.NewBatch() + p.renameNamespaceEntry(ctx, b, oldNameKey, desc) - desc.AddDrainingName(descpb.NameInfo{ - ParentID: keys.RootNamespaceID, - ParentSchemaID: keys.RootNamespaceID, - Name: oldName, - }) + // Write the updated database descriptor. if err := p.writeNonDropDatabaseChange(ctx, desc, stmt); err != nil { return err } + // Run the namespace update batch. return p.txn.Run(ctx, b) } diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index cacdbcabf44a..64ac41f38ac9 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -174,14 +173,10 @@ func (n *dropDatabaseNode) startExec(params runParams) error { return err } - n.dbDesc.AddDrainingName(descpb.NameInfo{ - ParentID: keys.RootNamespaceID, - ParentSchemaID: keys.RootNamespaceID, - Name: n.dbDesc.Name, - }) - n.dbDesc.State = descpb.DescriptorState_DROP + n.dbDesc.SetDropped() + b := p.txn.NewBatch() + p.dropNamespaceEntry(ctx, b, n.dbDesc) - b := &kv.Batch{} // Note that a job was already queued above. if err := p.writeDatabaseChangeToBatch(ctx, n.dbDesc, b); err != nil { return err diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index bc69c2b7f9ab..50209267e1fa 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -198,26 +199,41 @@ func (n *dropSchemaNode) startExec(params runParams) error { func (p *planner) dropSchemaImpl( ctx context.Context, parentDB *dbdesc.Mutable, sc *schemadesc.Mutable, ) error { - sc.AddDrainingName(descpb.NameInfo{ - ParentID: parentDB.ID, - ParentSchemaID: keys.RootNamespaceID, - Name: sc.Name, - }) - // TODO (rohany): This can be removed once RESTORE installs schemas into - // the parent database. - if parentDB.Schemas == nil { - parentDB.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) - } - parentDB.Schemas[sc.GetName()] = descpb.DatabaseDescriptor_SchemaInfo{ - ID: sc.GetID(), - Dropped: true, + + // Update parent database schemas mapping. + if p.execCfg.Settings.Version.IsActive(ctx, clusterversion.AvoidDrainingNames) { + delete(parentDB.Schemas, sc.GetName()) + } else { + // TODO (rohany): This can be removed once RESTORE installs schemas into + // the parent database. + if parentDB.Schemas == nil { + parentDB.Schemas = make(map[string]descpb.DatabaseDescriptor_SchemaInfo) + } + parentDB.Schemas[sc.GetName()] = descpb.DatabaseDescriptor_SchemaInfo{ + ID: sc.GetID(), + Dropped: true, + } } - // Mark the descriptor as dropped. - sc.State = descpb.DescriptorState_DROP + + // Update the schema descriptor as dropped. + sc.SetDropped() + + // Populate namespace update batch. + b := p.txn.NewBatch() + p.dropNamespaceEntry(ctx, b, sc) + + // Remove any associated comments. if err := p.removeSchemaComment(ctx, sc.GetID()); err != nil { return err } - return p.writeSchemaDesc(ctx, sc) + + // Write the updated descriptor. + if err := p.writeSchemaDesc(ctx, sc); err != nil { + return err + } + + // Run the namespace update batch. + return p.txn.Run(ctx, b) } func (p *planner) createDropSchemaJob( diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index b5a8873827e2..61cc96c8ac4c 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -457,14 +457,15 @@ func (p *planner) initiateDropTable( return err } - tableDesc.State = descpb.DescriptorState_DROP - - // Queue up name for draining. - tableDesc.AddDrainingName(descpb.NameInfo{ - ParentID: tableDesc.GetParentID(), - ParentSchemaID: tableDesc.GetParentSchemaID(), - Name: tableDesc.GetName(), - }) + // Actually mark table descriptor as dropped. + tableDesc.SetDropped() + + // Delete namespace entry for table. + b := p.txn.NewBatch() + p.dropNamespaceEntry(ctx, b, tableDesc) + if err := p.txn.Run(ctx, b); err != nil { + return err + } // For this table descriptor, mark all previous jobs scheduled for schema changes as successful // and delete them from the schema change job cache. diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go index d6943fcd1d45..34a962c2c5a9 100644 --- a/pkg/sql/drop_test.go +++ b/pkg/sql/drop_test.go @@ -1048,11 +1048,6 @@ CREATE TABLE test.t(a INT PRIMARY KEY); t.Fatal("table should be invisible through SHOW TABLES") } - // Check that CREATE TABLE with the same name returns a proper error. - if _, err := db.Exec(`CREATE TABLE test.t(a INT PRIMARY KEY)`); !testutils.IsError(err, `table "t" is being dropped, try again later`) { - t.Fatal(err) - } - // Check that DROP TABLE with the same name returns a proper error. if _, err := db.Exec(`DROP TABLE test.t`); !testutils.IsError(err, `relation "test.t" does not exist`) { t.Fatal(err) diff --git a/pkg/sql/drop_type.go b/pkg/sql/drop_type.go index 4db0a44bba46..a6ea53cbc35a 100644 --- a/pkg/sql/drop_type.go +++ b/pkg/sql/drop_type.go @@ -250,15 +250,17 @@ func (p *planner) dropTypeImpl( return errors.Errorf("type %q is already being dropped", typeDesc.Name) } - // Add a draining name. - typeDesc.AddDrainingName(descpb.NameInfo{ - ParentID: typeDesc.ParentID, - ParentSchemaID: typeDesc.ParentSchemaID, - Name: typeDesc.Name, - }) - // Actually mark the type as dropped. - typeDesc.State = descpb.DescriptorState_DROP + typeDesc.SetDropped() + + // Delete namespace entry for type. + b := p.txn.NewBatch() + p.dropNamespaceEntry(ctx, b, typeDesc) + if err := p.txn.Run(ctx, b); err != nil { + return err + } + + // Write updated type descriptor. if queueJob { return p.writeTypeSchemaChange(ctx, typeDesc, jobDesc) } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_type b/pkg/sql/logictest/testdata/logic_test/alter_type index eb84429e23c1..96e4c6a401bc 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_type +++ b/pkg/sql/logictest/testdata/logic_test/alter_type @@ -25,10 +25,6 @@ SELECT 'hi'::renameagain ---- hi -# newname is draining, so it should not be available for use. -statement error pq: type \"newname\" already exists -ALTER TYPE renameagain RENAME TO newname - statement ok ROLLBACK @@ -53,7 +49,6 @@ hi statement ok COMMIT -# newname should be available for use after draining. statement ok ALTER TYPE new__name RENAME TO newname diff --git a/pkg/sql/logictest/testdata/logic_test/database b/pkg/sql/logictest/testdata/logic_test/database index 3c7edf335996..21f692ef16a9 100644 --- a/pkg/sql/logictest/testdata/logic_test/database +++ b/pkg/sql/logictest/testdata/logic_test/database @@ -219,27 +219,8 @@ user testuser statement error permission denied to create database CREATE DATABASE privs -# Unit test for #61150 user root -statement ok -CREATE DATABASE d1 - -statement ok -BEGIN - -statement ok -DROP DATABASE d1 - -statement error pgcode 55000 database "d1" is being dropped, try again later -CREATE DATABASE IF NOT EXISTS d1 - -statement ok -END - -statement ok -DROP DATABASE d1 - subtest missing-db-error-issue-68060 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/drop_type b/pkg/sql/logictest/testdata/logic_test/drop_type index a79ba381cf8e..ab3a5ca9f434 100644 --- a/pkg/sql/logictest/testdata/logic_test/drop_type +++ b/pkg/sql/logictest/testdata/logic_test/drop_type @@ -41,17 +41,6 @@ CREATE TYPE t AS ENUM ('hello') statement error pq: \"_t\" is an implicit array type and cannot be modified DROP TYPE _t -# The dropped type name should be draining in the dropping transaction. -statement ok -BEGIN; -DROP TYPE t - -statement error pq: type \"test.public.t\" already exists -CREATE TYPE t AS ENUM () - -statement ok -ROLLBACK - # Now check all of the fun cases around object dependencies. # Test a simple column dependency. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/rename_database b/pkg/sql/logictest/testdata/logic_test/rename_database index 6db21b15db52..98cde8f1e8a4 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_database +++ b/pkg/sql/logictest/testdata/logic_test/rename_database @@ -237,67 +237,3 @@ DROP TABLE db2.a; CREATE TABLE db2.a (a int default nextval('a_seq') + nextval(' statement ok USE defaultdb; DROP DATABASE db1 CASCADE - -# Test that names cannot be reused within the same transaction. - -statement ok -CREATE DATABASE foo; CREATE DATABASE bar - -# rename foo -> baz, create foo - -statement ok -BEGIN - -statement ok -ALTER DATABASE foo RENAME TO baz - -statement error pgcode 42P04 database "foo" already exists -CREATE DATABASE foo - -statement ok -ROLLBACK - -# drop foo, rename bar -> foo - -statement ok -BEGIN - -statement ok -DROP DATABASE foo - -statement error pgcode 42P04 the new database name "foo" already exists -ALTER DATABASE bar RENAME TO foo - -statement ok -ROLLBACK - -# rename foo -> baz, rename bar -> foo - -statement ok -BEGIN - -statement ok -ALTER DATABASE foo RENAME TO baz - -statement error pgcode 42P04 the new database name "foo" already exists -ALTER DATABASE bar RENAME TO foo - -statement ok -ROLLBACK - -# drop foo, create foo - -statement ok -BEGIN - -statement ok -DROP DATABASE foo - -statement error pgcode 42P04 database "foo" already exists -CREATE DATABASE foo - -statement ok -ROLLBACK - -statement ok -DROP DATABASE foo; DROP DATABASE bar diff --git a/pkg/sql/logictest/testdata/logic_test/rename_table b/pkg/sql/logictest/testdata/logic_test/rename_table index d0ba3c248154..d035d1fa6f59 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_table +++ b/pkg/sql/logictest/testdata/logic_test/rename_table @@ -179,38 +179,6 @@ COMMIT statement ok INSERT INTO d.kv (k,v) VALUES ('c', 'd') -# A table rename disallows the use of the old name -statement ok -BEGIN - -statement ok -ALTER TABLE d.kv RENAME TO d.kv2 - -statement ok -INSERT INTO d.kv2 (k,v) VALUES ('e', 'f') - -statement error pgcode 42P01 relation \"d.kv\" does not exist -INSERT INTO d.kv (k,v) VALUES ('g', 'h') - -statement ok -ROLLBACK - -# A database rename disallows the use of the old name. -statement ok -BEGIN - -statement ok -ALTER DATABASE d RENAME TO dnew - -statement ok -INSERT INTO dnew.kv (k,v) VALUES ('e', 'f') - -statement error pgcode 42P01 relation \"d.kv\" does not exist -INSERT INTO d.kv (k,v) VALUES ('g', 'h') - -statement ok -ROLLBACK - # Check that on a rollback a database name cannot be used. statement ok BEGIN diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 3b39bff032ef..2595955529b0 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -154,11 +154,6 @@ ALTER SCHEMA myschema RENAME TO another_schema statement ok ALTER SCHEMA another_schema RENAME TO another_one -# The names should be draining, so we can't use the old ones -# anymore. -statement error pq: schema "myschema" already exists -ALTER SCHEMA another_one RENAME TO myschema - statement ok ROLLBACK @@ -219,20 +214,6 @@ CREATE TABLE dropped.t (x INT) statement ok ROLLBACK -# We shouldn't be able to create a conflicting schema name in the same transaction -# that it is dropped in. -statement ok -BEGIN - -statement ok -DROP SCHEMA dropped - -statement error pq: schema "dropped" already exists -CREATE SCHEMA dropped - -statement ok -ROLLBACK - # Test that we can drop multiple schemas as part of a single DROP statement. statement ok CREATE SCHEMA scdrop1; @@ -824,18 +805,6 @@ CREATE SCHEMA sc2 statement ok CREATE TYPE sc3 as enum('foo') -statement ok -BEGIN - -statement ok -DROP SCHEMA sc2 - -statement error pgcode 55000 schema "sc2" is being dropped, try again later -CREATE SCHEMA IF NOT EXISTS sc2 - -statement ok -END - # Regression test for #62920. The bug that motivated this test would populate # the schema entry in the database with the database's name rather than the # schemas. diff --git a/pkg/sql/logictest/testdata/logic_test/sequences b/pkg/sql/logictest/testdata/logic_test/sequences index b5c98f242a90..baf95e02587a 100644 --- a/pkg/sql/logictest/testdata/logic_test/sequences +++ b/pkg/sql/logictest/testdata/logic_test/sequences @@ -1713,7 +1713,6 @@ SELECT nextval('cached_lower_bound_test_2'); ---- 2 - query I SELECT nextval('cached_lower_bound_test_2'); ---- @@ -1761,58 +1760,9 @@ SELECT last_value FROM cached_lower_bound_test_2; statement ok DROP SEQUENCE cached_lower_bound_test_2; - -# Unit test for #60737 - -statement ok -CREATE SEQUENCE s1 MINVALUE -4 START WITH -2 CACHE 5 INCREMENT BY -2; - -statement ok -CREATE TABLE s2 (A int) - -# Case 1: Both s1's are sequences and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP SEQUENCE s1 - -statement error pgcode 55000 sequence "s1" is being dropped, try again later -CREATE SEQUENCE IF NOT EXISTS s1 MINVALUE -4 START WITH -2 CACHE 5 INCREMENT BY -2; - -statement ok -END - -# Case 2: Both s2's are different types and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP TABLE s2 - -statement error pgcode 42809 "s2" is not a sequence -CREATE SEQUENCE IF NOT EXISTS s2 MINVALUE -4 START WITH -2 CACHE 5 INCREMENT BY -2; - -statement ok -END - -# Case 3: Both s2's are a different type -statement ok -BEGIN - -statement error pgcode 42809 "s2" is not a sequence -CREATE SEQUENCE IF NOT EXISTS s2 MINVALUE -4 START WITH -2 CACHE 5 INCREMENT BY -2; - -statement ok -END - -statement ok -DROP SEQUENCE s1 - -statement ok -DROP TABLE s2 - # Validate that cross DB sequences are detected by internal tables +subtest cross_db_sequences + statement ok CREATE DATABASE db3; diff --git a/pkg/sql/logictest/testdata/logic_test/set_schema b/pkg/sql/logictest/testdata/logic_test/set_schema index a61a9583d909..f5035bfd1e31 100644 --- a/pkg/sql/logictest/testdata/logic_test/set_schema +++ b/pkg/sql/logictest/testdata/logic_test/set_schema @@ -386,9 +386,6 @@ ALTER TYPE s1.typ3 SET SCHEMA s2 statement ok SELECT 'hello'::s2.typ3 -statement error pq: type "s1.typ3" does not exist -SELECT 'hello'::s1.typ3 - statement ok ROLLBACK diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 96a29d7e7f98..f74f518501aa 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -854,55 +854,3 @@ query I SELECT estimated_row_count FROM [SHOW TABLES from rowtest] where table_name = 't1' ---- 1000 - -user root - -# Unit test for #60737 - -statement ok -CREATE TABLE t1 (a INT PRIMARY KEY, b INT) - -statement ok -CREATE TYPE t2 as enum('foo') - -# Case 1: Both t1's are tables and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP TABLE t1 - -statement error pgcode 55000 table "t1" is being dropped, try again later -CREATE TABLE IF NOT EXISTS t1 (a INT PRIMARY KEY, b INT) - -statement ok -END - -# Case 2: Both t2's are different types and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP TYPE t2 - -statement error pgcode 42809 "t2" is not a table -CREATE TABLE IF NOT EXISTS t2 (a INT PRIMARY KEY, b INT) - -statement ok -END - -# Case 3: Both s2's are a different type -statement ok -BEGIN - -statement error pgcode 42809 "t2" is not a table -CREATE TABLE IF NOT EXISTS t2 (a INT PRIMARY KEY, b INT) - -statement ok -END - -statement ok -DROP table t1 - -statement ok -DROP TYPE t2 diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index f635c7bb23f9..49c67195930f 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -845,64 +845,6 @@ CREATE VIEW db2.v1 AS SELECT a+b FROM db1.public.ab statement ok CREATE VIEW db2.v2 AS SELECT a+b+c+d FROM cd, db1.public.ab -# Unit test for #60737 -statement ok -CREATE TABLE t (a INT PRIMARY KEY, b INT) - -statement ok -INSERT INTO t VALUES (1, 99), (2, 98), (3, 97) - -statement ok -CREATE VIEW v3 AS SELECT a, b FROM t - -statement ok -CREATE TYPE v4 as enum('foo') - -# Case 1: Both v3's are views and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP VIEW v3 - -statement error pgcode 55000 view "v3" is being dropped, try again later -CREATE VIEW IF NOT EXISTS v3 AS SELECT a, b FROM t - -statement ok -END - -# Case 2: Both v4's are different types and the old one is being dropped. -statement ok -BEGIN - -statement ok -DROP TYPE v4 - -statement error pgcode 42809 "v4" is not a view -CREATE VIEW IF NOT EXISTS v4 AS SELECT a, b FROM t - -statement ok -END - -# Case 3: Both v4's are a different type -statement ok -BEGIN - -statement error pgcode 42809 "v4" is not a view -CREATE VIEW IF NOT EXISTS v4 AS SELECT a, b FROM t - -statement ok -END - -statement ok -DROP VIEW v3 - -statement ok -DROP TYPE v4 - -statement ok -DROP TABLE t - # Validate that cross DB views are detected by internal tables statement ok diff --git a/pkg/sql/name_util.go b/pkg/sql/name_util.go new file mode 100644 index 000000000000..34239c742bec --- /dev/null +++ b/pkg/sql/name_util.go @@ -0,0 +1,65 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func (p *planner) dropNamespaceEntry( + ctx context.Context, b *kv.Batch, desc catalog.MutableDescriptor, +) { + // Delete current namespace entry. + deleteNamespaceEntryAndMaybeAddDrainingName(ctx, b, p, desc, desc) +} + +func (p *planner) renameNamespaceEntry( + ctx context.Context, b *kv.Batch, oldNameKey catalog.NameKey, desc catalog.MutableDescriptor, +) { + // Delete old namespace entry. + deleteNamespaceEntryAndMaybeAddDrainingName(ctx, b, p, oldNameKey, desc) + + // Write new namespace entry. + marshalledKey := catalogkeys.EncodeNameKey(p.ExecCfg().Codec, desc) + if p.extendedEvalCtx.Tracing.KVTracingEnabled() { + log.VEventf(ctx, 2, "CPut %s -> %d", marshalledKey, desc.GetID()) + } + b.CPut(marshalledKey, desc.GetID(), nil) +} + +func deleteNamespaceEntryAndMaybeAddDrainingName( + ctx context.Context, + b *kv.Batch, + p *planner, + nameKeyToDelete catalog.NameKey, + desc catalog.MutableDescriptor, +) { + if !p.execCfg.Settings.Version.IsActive(ctx, clusterversion.AvoidDrainingNames) { + desc.AddDrainingName(descpb.NameInfo{ + ParentID: nameKeyToDelete.GetParentID(), + ParentSchemaID: nameKeyToDelete.GetParentSchemaID(), + Name: nameKeyToDelete.GetName(), + }) + return + } + marshalledKey := catalogkeys.EncodeNameKey(p.ExecCfg().Codec, nameKeyToDelete) + if p.extendedEvalCtx.Tracing.KVTracingEnabled() { + log.VEventf(ctx, 2, "Del %s", marshalledKey) + } + b.Del(marshalledKey) +} diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic index 4442f9c6fcdb..06936e0c8913 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace_nonmetamorphic @@ -168,7 +168,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -batch flow coordinator Put /Table/3/1/55/2/1 -> table: draining_names: parent_id:53 unexposed_parent_schema_id:29 columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" > next_index_id:2 privileges: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Del /NamespaceTable/30/1/53/29/"kv2"/4/1 +batch flow coordinator Put /Table/3/1/55/2/1 -> table: parent_id:53 unexposed_parent_schema_id:29 columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" > next_index_id:2 privileges: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 statement ok @@ -219,7 +220,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -batch flow coordinator Put /Table/3/1/54/2/1 -> table: draining_names: parent_id:53 unexposed_parent_schema_id:29 columns: TypeMeta: > nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" > next_index_id:3 privileges: users: owner_proto:"root" version:2 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 gc_mutations: create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > +batch flow coordinator Del /NamespaceTable/30/1/53/29/"kv"/4/1 +batch flow coordinator Put /Table/3/1/54/2/1 -> table: parent_id:53 unexposed_parent_schema_id:29 columns: TypeMeta: > nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: TypeMeta: > nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" > next_index_id:3 privileges: users: owner_proto:"root" version:2 > next_mutation_id:3 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false new_schema_change_job_id:0 drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 gc_mutations: create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false > exec stmt rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index f866c46ae728..cca2d9850b29 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -13,9 +13,7 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -104,7 +102,11 @@ func (n *renameTableNode) startExec(params runParams) error { ctx := params.ctx tableDesc := n.tableDesc oldTn := n.oldTn - prevDBID := tableDesc.ParentID + oldNameKey := descpb.NameInfo{ + ParentID: tableDesc.GetParentID(), + ParentSchemaID: tableDesc.GetParentSchemaID(), + Name: tableDesc.GetName(), + } var targetDbDesc catalog.DatabaseDescriptor var targetSchemaDesc catalog.SchemaDescriptor @@ -232,24 +234,19 @@ func (n *renameTableNode) startExec(params runParams) error { return err } - descID := tableDesc.GetID() - parentSchemaID := tableDesc.GetParentSchemaID() - - renameDetails := descpb.NameInfo{ - ParentID: prevDBID, - ParentSchemaID: parentSchemaID, - Name: oldTn.Table()} - tableDesc.AddDrainingName(renameDetails) + // Populate namespace update batch. + b := p.txn.NewBatch() + p.renameNamespaceEntry(ctx, b, oldNameKey, tableDesc) + // Write the updated table descriptor. if err := p.writeSchemaChange( ctx, tableDesc, descpb.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann()), ); err != nil { return err } - newTbKey := catalogkeys.NewNameKeyComponents(targetDbDesc.GetID(), tableDesc.GetParentSchemaID(), newTn.Table()) - - if err := p.writeNameKey(ctx, newTbKey, descID); err != nil { + // Run the namespace update batch. + if err := p.txn.Run(ctx, b); err != nil { return err } @@ -541,15 +538,3 @@ func (n *renameTableNode) checkForCrossDbReferences( } return nil } - -// writeNameKey writes a name key to a batch and runs the batch. -func (p *planner) writeNameKey(ctx context.Context, nameKey catalog.NameKey, ID descpb.ID) error { - marshalledKey := catalogkeys.EncodeNameKey(p.ExecCfg().Codec, nameKey) - b := &kv.Batch{} - if p.extendedEvalCtx.Tracing.KVTracingEnabled() { - log.VEventf(ctx, 2, "CPut %s -> %d", marshalledKey, ID) - } - b.CPut(marshalledKey, ID, nil) - - return p.txn.Run(ctx, b) -} diff --git a/pkg/sql/rename_test.go b/pkg/sql/rename_test.go index d2a3cea9add1..3872d7a2f985 100644 --- a/pkg/sql/rename_test.go +++ b/pkg/sql/rename_test.go @@ -12,7 +12,6 @@ package sql import ( "context" - "sync" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -89,9 +88,6 @@ func TestRenameTable(t *testing.T) { // Test that a SQL txn that resolved a name can keep resolving that name during // its lifetime even after the table has been renamed. -// Also tests that the name of a renamed table cannot be reused until everybody -// has stopped using it. Otherwise, we'd have different transactions in the -// systems using a single name for different tables. // Also tests that the old name cannot be used by node that doesn't have a lease // on the old version even while the name mapping still exists. func TestTxnCanStillResolveOldName(t *testing.T) { @@ -191,15 +187,6 @@ CREATE TABLE test.t (a INT PRIMARY KEY); t.Fatal(err) } - // Check that the name cannot be reused while somebody still has a lease on - // the old one (the mechanism for ensuring this is that the entry for the old - // name is not deleted from the database until the async schema changer checks - // that there's no more leases on the old version). - if _, err := db.Exec("CREATE TABLE test.t (a INT PRIMARY KEY)"); !testutils.IsError( - err, `relation "test.public.t" already exists`) { - t.Fatal(err) - } - if err := txn.Commit(); err != nil { t.Fatal(err) } @@ -281,12 +268,6 @@ CREATE TABLE test.t (a INT PRIMARY KEY); if _, err := txn.Exec("SELECT * FROM test.t"); err != nil { t.Fatal(err) } - // Check that we cannot use the old name. - if _, err := txn.Exec(` -SELECT * FROM test.t2 -`); !testutils.IsError(err, "relation \"test.t2\" does not exist") { - t.Fatalf("err = %v", err) - } if err := txn.Rollback(); err != nil { t.Fatal(err) } @@ -339,95 +320,3 @@ CREATE TABLE test.t (a INT PRIMARY KEY); t.Fatal(err) } } - -// Tests that a RENAME while a name is being drained will result in the -// table version being incremented again, implying that all old names -// are drained correctly. The new RENAME will succeed with -// all old names drained. -func TestRenameDuringDrainingName(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - // two channels that signal the start of the second rename - // and the end of the second rename. - startRename := make(chan interface{}) - finishRename := make(chan interface{}) - serverParams := base.TestServerArgs{ - Knobs: base.TestingKnobs{ - SQLSchemaChanger: &SchemaChangerTestingKnobs{ - OldNamesDrainedNotification: func() { - if startRename != nil { - // Run second rename. - start := startRename - startRename = nil - close(start) - <-finishRename - } - }, - // Don't run the schema changer for the second RENAME so that we can be - // sure that the first schema changer runs both schema changes. This - // behavior is due to the fact that the schema changer for the first job - // will process all the draining names on the table descriptor, - // including the ones queued up by the second job. It's not ideal since - // we would like jobs to manage their own state without interference - // from other jobs, but that's how schema change jobs work right now. - SchemaChangeJobNoOp: func() bool { - return startRename == nil - }, - }, - }} - - s, db, kvDB := serverutils.StartServer(t, serverParams) - defer s.Stopper().Stop(context.Background()) - - sql := ` -CREATE DATABASE test; -CREATE TABLE test.t (a INT PRIMARY KEY); -` - _, err := db.Exec(sql) - if err != nil { - t.Fatal(err) - } - - tableDesc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") - // The expected version will be the result of two increments for the two - // schema changes and one increment for signaling of the completion of the - // drain. See the above comment for an explanation of why there's only one - // expected version update for draining names. - expectedVersion := tableDesc.GetVersion() + 3 - - // Concurrently, rename the table. - start := startRename - var wg sync.WaitGroup - wg.Add(1) - go func() { - if _, err := db.Exec("ALTER TABLE test.t RENAME TO test.t2"); err != nil { - t.Error(err) - } - wg.Done() - }() - - <-start - if _, err := db.Exec("ALTER TABLE test.t2 RENAME TO test.t3"); err != nil { - t.Fatal(err) - } - close(finishRename) - - wg.Wait() - - // Table rename to t3 was successful. - tableDesc = catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t3") - if version := tableDesc.GetVersion(); expectedVersion != version { - t.Fatalf("version mismatch: expected = %d, current = %d", expectedVersion, version) - } - - // Old names are gone. - if _, err := db.Exec("SELECT * FROM test.t"); !testutils.IsError( - err, `relation "test.t" does not exist`) { - t.Fatal(err) - } - if _, err := db.Exec("SELECT * FROM test.t2"); !testutils.IsError( - err, `relation "test.t2" does not exist`) { - t.Fatal(err) - } -} diff --git a/pkg/sql/reparent_database.go b/pkg/sql/reparent_database.go index 3caaabbf6125..bbe161ce6aae 100644 --- a/pkg/sql/reparent_database.go +++ b/pkg/sql/reparent_database.go @@ -13,7 +13,6 @@ package sql import ( "context" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -173,6 +172,11 @@ func (n *reparentDatabaseNode) startExec(params runParams) error { return err } if found { + oldNameKey := descpb.NameInfo{ + ParentID: desc.GetParentID(), + ParentSchemaID: desc.GetParentSchemaID(), + Name: desc.GetName(), + } // Remap the ID's on the table. tbl, ok := desc.(*tabledesc.Mutable) if !ok { @@ -207,15 +211,9 @@ func (n *reparentDatabaseNode) startExec(params runParams) error { ) } - tbl.AddDrainingName(descpb.NameInfo{ - ParentID: tbl.ParentID, - ParentSchemaID: tbl.GetParentSchemaID(), - Name: tbl.Name, - }) tbl.ParentID = n.newParent.ID tbl.UnexposedParentSchemaID = schema.GetID() - objKey := catalogkeys.EncodeNameKey(codec, tbl) - b.CPut(objKey, tbl.GetID(), nil /* expected */) + p.renameNamespaceEntry(ctx, b, oldNameKey, tbl) if err := p.writeSchemaChange(ctx, tbl, descpb.InvalidMutationID, tree.AsStringWithFQNames(n.n, params.Ann())); err != nil { return err } @@ -242,20 +240,19 @@ func (n *reparentDatabaseNode) startExec(params runParams) error { if !found { continue } + oldNameKey := descpb.NameInfo{ + ParentID: desc.GetParentID(), + ParentSchemaID: desc.GetParentSchemaID(), + Name: desc.GetName(), + } // Remap the ID's on the type. typ, ok := desc.(*typedesc.Mutable) if !ok { return errors.AssertionFailedf("%q was not a Mutable", objName.Object()) } - typ.AddDrainingName(descpb.NameInfo{ - ParentID: typ.ParentID, - ParentSchemaID: typ.ParentSchemaID, - Name: typ.Name, - }) typ.ParentID = n.newParent.ID typ.ParentSchemaID = schema.GetID() - objKey := catalogkeys.EncodeNameKey(codec, typ) - b.CPut(objKey, typ.ID, nil /* expected */) + p.renameNamespaceEntry(ctx, b, oldNameKey, typ) if err := p.writeTypeSchemaChange(ctx, typ, tree.AsStringWithFQNames(n.n, params.Ann())); err != nil { return err } @@ -268,12 +265,9 @@ func (n *reparentDatabaseNode) startExec(params runParams) error { // This command can only be run when database leasing is supported, so we don't // have to handle the case where it isn't. - n.db.AddDrainingName(descpb.NameInfo{ - ParentID: keys.RootNamespaceID, - ParentSchemaID: keys.RootNamespaceID, - Name: n.db.Name, - }) - n.db.State = descpb.DescriptorState_DROP + p.dropNamespaceEntry(ctx, b, n.db) + + n.db.SetDropped() if err := p.writeDatabaseChangeToBatch(ctx, n.db, b); err != nil { return err } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 2ae8c1862ba7..62029258b9ee 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -3648,13 +3648,6 @@ INSERT INTO t.kv VALUES ('a', 'b'); secondStmt string expectedErr string }{ - // DROP TABLE followed by CREATE TABLE case. - { - name: `drop-create`, - firstStmt: `DROP TABLE t.kv`, - secondStmt: `CREATE TABLE t.kv (k CHAR PRIMARY KEY, v CHAR)`, - expectedErr: `table "kv" is being dropped, try again later`, - }, // schema change followed by another statement works. { name: `createindex-insert`, @@ -6518,61 +6511,6 @@ CREATE INDEX i ON t.test (a) WHERE b > 2 } } -// TestDrainingNamesCannotBeResolved tests that during the draining names state -// for renamed descriptors, old names cannot be used via the uncached name -// resolution path. -func TestDrainingNamesCannotBeResolved(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - params, _ := tests.CreateTestServerParams() - params.Knobs = base.TestingKnobs{ - // Don't drain names. This also means that we don't wait for leases to - // drain before returning, which is fine since we're testing the non- - // leased name resolution path. - SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ - SchemaChangeJobNoOp: func() bool { return true }, - }, - SQLTypeSchemaChanger: &sql.TypeSchemaChangerTestingKnobs{ - TypeSchemaChangeJobNoOp: func() bool { return true }, - }, - } - - ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, params) - defer s.Stopper().Stop(ctx) - sqlRun := sqlutils.MakeSQLRunner(sqlDB) - - // Create descriptors and rename them so that the old names get stuck as - // draining names. - sqlRun.Exec(t, ` -CREATE TABLE tbl(); -ALTER TABLE tbl RENAME TO tbl2; - -CREATE TYPE typ AS ENUM(); -ALTER TYPE typ RENAME TO typ2; - -CREATE SCHEMA sc; -ALTER SCHEMA sc RENAME TO sc2; - -CREATE DATABASE db; -ALTER DATABASE db RENAME TO db2; -`) - - // Ensure that the old namespace entries still exist. - sqlRun.CheckQueryResults(t, `SELECT count(*) FROM system.namespace WHERE name = 'tbl'`, [][]string{{"1"}}) - sqlRun.CheckQueryResults(t, `SELECT count(*) FROM system.namespace WHERE name = 'typ'`, [][]string{{"1"}}) - sqlRun.CheckQueryResults(t, `SELECT count(*) FROM system.namespace WHERE name = 'sc'`, [][]string{{"1"}}) - sqlRun.CheckQueryResults(t, `SELECT count(*) FROM system.namespace WHERE name = 'db'`, [][]string{{"1"}}) - - // Test uncached name resolution by attempting schema changes. As of 20.2 we - // have some internal inconsistency in error messages. - sqlRun.ExpectErr(t, `relation "tbl" does not exist`, `ALTER TABLE tbl RENAME TO tbl3`) - sqlRun.ExpectErr(t, `type "typ" does not exist`, `ALTER TYPE typ RENAME TO typ3`) - sqlRun.ExpectErr(t, `unknown schema "sc"`, `ALTER SCHEMA sc RENAME TO sc3`) - sqlRun.ExpectErr(t, `database "db" does not exist`, `ALTER DATABASE db RENAME TO db3`) -} - // TestAddingTableResolution tests that table names cannot be resolved in the // adding state. func TestAddingTableResolution(t *testing.T) { diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 3deb61d77fd0..007359720d00 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -213,6 +213,12 @@ func cleanupSessionTempObjects( } // cleanupSchemaObjects removes all objects that is located within a dbID and schema. +// +// TODO(postamar): properly use descsCol +// We're currently unable to leverage descsCol properly because we run DROP +// statements in the transaction which cause descsCol's cached state to become +// invalid. We should either drop all objects programmatically via descsCol's +// API or avoid it entirely. func cleanupSchemaObjects( ctx context.Context, settings *cluster.Settings, @@ -227,7 +233,7 @@ func cleanupSchemaObjects( if err != nil { return err } - tbNames, _, err := descsCol.GetObjectNamesAndIDs( + tbNames, tbIDs, err := descsCol.GetObjectNamesAndIDs( ctx, txn, dbDesc, @@ -250,10 +256,8 @@ func cleanupSchemaObjects( tblDescsByID := make(map[descpb.ID]catalog.TableDescriptor, len(tbNames)) tblNamesByID := make(map[descpb.ID]tree.TableName, len(tbNames)) - for _, tbName := range tbNames { - flags := tree.ObjectLookupFlagsWithRequired() - flags.AvoidCached = true - _, desc, err := descsCol.GetImmutableTableByName(ctx, txn, &tbName, flags) + for i, tbName := range tbNames { + desc, err := catalogkv.MustGetTableDescByID(ctx, txn, codec, tbIDs[i]) if err != nil { return err }