Skip to content

Commit

Permalink
descs: move uncommitted descriptors to their own layer
Browse files Browse the repository at this point in the history
Previously, uncommitted descriptors were maintained as part of the
kvDescriptors layer in descs.Collection. This commit moves them to their
own layer, which helps to further unify the code paths for descriptor
lookups in the Collection. Pre-txn validation has also been improved,
reducing the number of round trips to storage.

Release note: None
  • Loading branch information
Marius Posta committed Aug 18, 2021
1 parent cee3378 commit b94d1f2
Show file tree
Hide file tree
Showing 12 changed files with 666 additions and 363 deletions.
110 changes: 55 additions & 55 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
@@ -1,57 +1,57 @@
exp,benchmark
19,AlterRole/alter_role_with_1_option
20,AlterRole/alter_role_with_2_options
25,AlterRole/alter_role_with_3_options
17,AlterTableAddCheckConstraint/alter_table_add_1_check_constraint
17,AlterTableAddCheckConstraint/alter_table_add_2_check_constraints
17,AlterTableAddCheckConstraint/alter_table_add_3_check_constraints
17,AlterTableAddColumn/alter_table_add_1_column
17,AlterTableAddColumn/alter_table_add_2_columns
17,AlterTableAddColumn/alter_table_add_3_columns
22,AlterTableAddForeignKey/alter_table_add_1_foreign_key
27,AlterTableAddForeignKey/alter_table_add_2_foreign_keys
32,AlterTableAddForeignKey/alter_table_add_3_foreign_keys
22,AlterTableAddForeignKey/alter_table_add_foreign_key_with_3_columns
15,AlterRole/alter_role_with_1_option
16,AlterRole/alter_role_with_2_options
21,AlterRole/alter_role_with_3_options
15,AlterTableAddCheckConstraint/alter_table_add_1_check_constraint
15,AlterTableAddCheckConstraint/alter_table_add_2_check_constraints
15,AlterTableAddCheckConstraint/alter_table_add_3_check_constraints
15,AlterTableAddColumn/alter_table_add_1_column
15,AlterTableAddColumn/alter_table_add_2_columns
15,AlterTableAddColumn/alter_table_add_3_columns
20,AlterTableAddForeignKey/alter_table_add_1_foreign_key
25,AlterTableAddForeignKey/alter_table_add_2_foreign_keys
30,AlterTableAddForeignKey/alter_table_add_3_foreign_keys
20,AlterTableAddForeignKey/alter_table_add_foreign_key_with_3_columns
20,AlterTableConfigureZone/alter_table_configure_zone_5_replicas
20,AlterTableConfigureZone/alter_table_configure_zone_7_replicas_
20,AlterTableConfigureZone/alter_table_configure_zone_ranges
20,AlterTableDropColumn/alter_table_drop_1_column
22,AlterTableDropColumn/alter_table_drop_2_columns
24,AlterTableDropColumn/alter_table_drop_3_columns
18,AlterTableDropConstraint/alter_table_drop_1_check_constraint
19,AlterTableDropConstraint/alter_table_drop_2_check_constraints
20,AlterTableDropConstraint/alter_table_drop_3_check_constraints
18,AlterTableDropColumn/alter_table_drop_1_column
20,AlterTableDropColumn/alter_table_drop_2_columns
22,AlterTableDropColumn/alter_table_drop_3_columns
16,AlterTableDropConstraint/alter_table_drop_1_check_constraint
17,AlterTableDropConstraint/alter_table_drop_2_check_constraints
18,AlterTableDropConstraint/alter_table_drop_3_check_constraints
12-13,AlterTableSplit/alter_table_split_at_1_value
18-19,AlterTableSplit/alter_table_split_at_2_values
24-25,AlterTableSplit/alter_table_split_at_3_values
8,AlterTableUnsplit/alter_table_unsplit_at_1_value
10,AlterTableUnsplit/alter_table_unsplit_at_2_values
12,AlterTableUnsplit/alter_table_unsplit_at_3_values
21,CreateRole/create_role_with_1_option
23,CreateRole/create_role_with_2_options
24,CreateRole/create_role_with_3_options
22,CreateRole/create_role_with_no_options
21,DropDatabase/drop_database_0_tables
28,DropDatabase/drop_database_1_table
35,DropDatabase/drop_database_2_tables
42,DropDatabase/drop_database_3_tables
29,DropRole/drop_1_role
36,DropRole/drop_2_roles
43,DropRole/drop_3_roles
19,DropSequence/drop_1_sequence
26,DropSequence/drop_2_sequences
33,DropSequence/drop_3_sequences
21,DropTable/drop_1_table
29,DropTable/drop_2_tables
37,DropTable/drop_3_tables
22,DropView/drop_1_view
30,DropView/drop_2_views
38,DropView/drop_3_views
18,Grant/grant_all_on_1_table
20,Grant/grant_all_on_2_tables
22,Grant/grant_all_on_3_tables
21,GrantRole/grant_1_role
24,GrantRole/grant_2_roles
17,CreateRole/create_role_with_1_option
19,CreateRole/create_role_with_2_options
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
25,DropRole/drop_1_role
32,DropRole/drop_2_roles
39,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
16,Grant/grant_all_on_1_table
18,Grant/grant_all_on_2_tables
20,Grant/grant_all_on_3_tables
17,GrantRole/grant_1_role
20,GrantRole/grant_2_roles
2,ORMQueries/activerecord_type_introspection_query
2,ORMQueries/django_table_introspection_1_table
2,ORMQueries/django_table_introspection_4_tables
Expand All @@ -64,19 +64,19 @@ exp,benchmark
2,ORMQueries/pg_class
2,ORMQueries/pg_namespace
2,ORMQueries/pg_type
18,Revoke/revoke_all_on_1_table
20,Revoke/revoke_all_on_2_tables
22,Revoke/revoke_all_on_3_tables
20,RevokeRole/revoke_1_role
22,RevokeRole/revoke_2_roles
16,Revoke/revoke_all_on_1_table
18,Revoke/revoke_all_on_2_tables
20,Revoke/revoke_all_on_3_tables
16,RevokeRole/revoke_1_role
18,RevokeRole/revoke_2_roles
1,SystemDatabaseQueries/select_system.users_with_empty_database_Name
1,SystemDatabaseQueries/select_system.users_with_schema_Name
2,SystemDatabaseQueries/select_system.users_without_schema_Name
25,Truncate/truncate_1_column_0_rows
25,Truncate/truncate_1_column_1_row
25,Truncate/truncate_1_column_2_rows
25,Truncate/truncate_2_column_0_rows
25,Truncate/truncate_2_column_1_rows
25,Truncate/truncate_2_column_2_rows
23,Truncate/truncate_1_column_0_rows
23,Truncate/truncate_1_column_1_row
23,Truncate/truncate_1_column_2_rows
23,Truncate/truncate_2_column_0_rows
23,Truncate/truncate_2_column_1_rows
23,Truncate/truncate_2_column_2_rows
1,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
36 changes: 18 additions & 18 deletions pkg/ccl/benchccl/rttanalysisccl/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
exp,benchmark
23,AlterPrimaryRegion/alter_empty_database_alter_primary_region
24,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
23,AlterPrimaryRegion/alter_populated_database_alter_primary_region
25,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
22,AlterRegions/alter_empty_database_add_region
23,AlterRegions/alter_empty_database_drop_region
22,AlterRegions/alter_populated_database_add_region
23,AlterRegions/alter_populated_database_drop_region
23,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
23,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
43,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
43,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
24,AlterTableLocality/alter_from_global_to_rbr
26,AlterTableLocality/alter_from_global_to_regional_by_table
22,AlterTableLocality/alter_from_rbr_to_global
22,AlterTableLocality/alter_from_rbr_to_regional_by_table
26,AlterTableLocality/alter_from_regional_by_table_to_global
24,AlterTableLocality/alter_from_regional_by_table_to_rbr
21,AlterPrimaryRegion/alter_empty_database_alter_primary_region
22,AlterPrimaryRegion/alter_empty_database_set_initial_primary_region
21,AlterPrimaryRegion/alter_populated_database_alter_primary_region
23,AlterPrimaryRegion/alter_populated_database_set_initial_primary_region
20,AlterRegions/alter_empty_database_add_region
21,AlterRegions/alter_empty_database_drop_region
20,AlterRegions/alter_populated_database_add_region
21,AlterRegions/alter_populated_database_drop_region
21,AlterSurvivalGoals/alter_empty_database_from_region_to_zone
21,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
41,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
41,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
22,AlterTableLocality/alter_from_global_to_rbr
24,AlterTableLocality/alter_from_global_to_regional_by_table
20,AlterTableLocality/alter_from_rbr_to_global
20,AlterTableLocality/alter_from_rbr_to_regional_by_table
24,AlterTableLocality/alter_from_regional_by_table_to_global
22,AlterTableLocality/alter_from_regional_by_table_to_rbr
Original file line number Diff line number Diff line change
Expand Up @@ -47,16 +47,21 @@ func TestDeleteDeprecatedNamespaceDescriptorMigration(t *testing.T) {
})
defer tc.Stopper().Stop(ctx)

// Inject deprecated namespace table descriptor and namespace entries.
// Inject deprecated namespace table descriptors and namespace entries.
err := tc.Servers[0].DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
codec := keys.SystemSQLCodec
deprecated := *systemschema.NamespaceTable.TableDesc()
deprecated.ID = keys.DeprecatedNamespaceTableID
descProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &deprecated}}
deprecatedDescProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &deprecated}}
namespace2 := *systemschema.NamespaceTable.TableDesc()
namespace2.ID = keys.NamespaceTableID
namespace2.Name = `namespace2`
ns2DescProto := &descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &namespace2}}
b := txn.NewBatch()
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.DeprecatedNamespaceTableID), descProto)
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.DeprecatedNamespaceTableID), deprecatedDescProto)
namespaceKey := catalogkeys.MakePublicObjectNameKey(codec, keys.SystemDatabaseID, `namespace`)
b.Put(namespaceKey, keys.DeprecatedNamespaceTableID)
b.Put(catalogkeys.MakeDescMetadataKey(codec, keys.NamespaceTableID), ns2DescProto)
namespace2Key := catalogkeys.MakePublicObjectNameKey(codec, keys.SystemDatabaseID, `namespace2`)
b.Put(namespace2Key, keys.NamespaceTableID)
return txn.Run(ctx, b)
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/catalog/descs/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ go_library(
"temporary_descriptors.go",
"txn.go",
"type.go",
"uncommitted_descriptors.go",
"validate.go",
"virtual_descriptors.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs",
Expand Down
56 changes: 20 additions & 36 deletions pkg/sql/catalog/descs/collection.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand All @@ -53,6 +52,7 @@ func makeCollection(
virtual: makeVirtualDescriptors(virtualSchemas),
leased: makeLeasedDescriptors(leaseMgr),
synthetic: makeSyntheticDescriptors(),
uncommitted: makeUncommittedDescriptors(),
kv: makeKVDescriptors(codec),
temporary: makeTemporaryDescriptors(codec, sessionData),
}
Expand All @@ -75,6 +75,14 @@ type Collection struct {
// the transaction using them is complete.
leased leasedDescriptors

// Descriptors modified by the uncommitted transaction affiliated with this
// Collection. This allows a transaction to see its own modifications while
// bypassing the descriptor lease mechanism. The lease mechanism will have its
// own transaction to read the descriptor and will hang waiting for the
// uncommitted changes to the descriptor. These descriptors are local to this
// Collection and invisible to other transactions.
uncommitted uncommittedDescriptors

// A collection of descriptors which were read from the store.
kv kvDescriptors

Expand Down Expand Up @@ -149,6 +157,7 @@ func (tc *Collection) ReleaseLeases(ctx context.Context) {
// ReleaseAll calls ReleaseLeases.
func (tc *Collection) ReleaseAll(ctx context.Context) {
tc.ReleaseLeases(ctx)
tc.uncommitted.reset()
tc.kv.reset()
tc.synthetic.reset()
tc.deletedDescs = nil
Expand All @@ -157,13 +166,13 @@ func (tc *Collection) ReleaseAll(ctx context.Context) {
// HasUncommittedTables returns true if the Collection contains uncommitted
// tables.
func (tc *Collection) HasUncommittedTables() bool {
return tc.kv.hasUncommittedTables()
return tc.uncommitted.hasUncommittedTables()
}

// HasUncommittedTypes returns true if the Collection contains uncommitted
// types.
func (tc *Collection) HasUncommittedTypes() bool {
return tc.kv.hasUncommittedTypes()
return tc.uncommitted.hasUncommittedTypes()
}

// Satisfy the linter.
Expand All @@ -179,23 +188,7 @@ var _ = (*Collection).HasUncommittedTypes
// immutably will return a copy of the descriptor in the current state. A deep
// copy is performed in this call.
func (tc *Collection) AddUncommittedDescriptor(desc catalog.MutableDescriptor) error {
_, err := tc.kv.addUncommittedDescriptor(desc)
return err
}

// maybeRefreshCachedFieldsOnTypeDescriptor refreshes the cached fields on a
// Mutable if the given descriptor is a type descriptor and works as a pass
// through for all other descriptors. Mutable type descriptors are refreshed to
// reconstruct enumMetadata. This ensures that tables hydration following a
// type descriptor update (in the same txn) happens using the modified fields.
func maybeRefreshCachedFieldsOnTypeDescriptor(
desc catalog.MutableDescriptor,
) (catalog.MutableDescriptor, error) {
typeDesc, ok := desc.(catalog.TypeDescriptor)
if ok {
return typedesc.UpdateCachedFieldsOnModifiedMutable(typeDesc)
}
return desc, nil
return tc.uncommitted.checkIn(desc)
}

// ValidateOnWriteEnabled is the cluster setting used to enable or disable
Expand Down Expand Up @@ -238,27 +231,18 @@ func (tc *Collection) WriteDesc(
// undergone a schema change. Returns nil for no schema changes. The version
// returned for each schema change is ClusterVersion - 1, because that's the one
// that will be used when checking for table descriptor two version invariance.
func (tc *Collection) GetDescriptorsWithNewVersion() []lease.IDVersion {
return tc.kv.getDescriptorsWithNewVersion()
func (tc *Collection) GetDescriptorsWithNewVersion() (originalVersions []lease.IDVersion) {
_ = tc.uncommitted.iterateNewVersionByID(func(_ catalog.NameEntry, originalVersion lease.IDVersion) error {
originalVersions = append(originalVersions, originalVersion)
return nil
})
return originalVersions
}

// GetUncommittedTables returns all the tables updated or created in the
// transaction.
func (tc *Collection) GetUncommittedTables() (tables []catalog.TableDescriptor) {
return tc.kv.getUncommittedTables()
}

// ValidateUncommittedDescriptors validates all uncommitted descriptors.
// Validation includes cross-reference checks. Referenced descriptors are
// read from the store unless they happen to also be part of the uncommitted
// descriptor set. We purposefully avoid using leased descriptors as those may
// be one version behind, in which case it's possible (and legitimate) that
// those are missing back-references which would cause validation to fail.
func (tc *Collection) ValidateUncommittedDescriptors(ctx context.Context, txn *kv.Txn) error {
if tc.skipValidationOnWrite || !ValidateOnWriteEnabled.Get(&tc.settings.SV) {
return nil
}
return tc.kv.validateUncommittedDescriptors(ctx, txn)
return tc.uncommitted.getUncommittedTables()
}

func newMutableSyntheticDescriptorAssertionError(id descpb.ID) error {
Expand Down
Loading

0 comments on commit b94d1f2

Please sign in to comment.