Skip to content

Commit

Permalink
sql/catalog: always clone with the modification timestamp set
Browse files Browse the repository at this point in the history
Previously, the RunPostDeserializationChanges would always
copy descriptors even with our new copy on write semantics. This would
happen because the original copy never has the modification or creation
timestamps set. To address this, this patch modifies the builder to set
these timestamps within the original copy stored inside.

Release note: None
  • Loading branch information
fqazi committed Aug 2, 2024
1 parent 9e71096 commit e89a581
Show file tree
Hide file tree
Showing 7 changed files with 229 additions and 156 deletions.
1 change: 1 addition & 0 deletions pkg/sql/catalog/systemschema_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_test(
"//pkg/testutils/datapathutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/log/eventpb",
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/catalog/systemschema_test/systemschema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
Expand Down Expand Up @@ -116,6 +117,12 @@ func runTest(t *testing.T, path string, db *gosql.DB, execCfg *sql.ExecutorConfi
require.EqualValues(t, len(events), 1+meta.NumRecords, "unexpected record count")
for _, event := range events[1:] {
ev, ok := event.(*eventpb.SchemaDescriptor)
// Always clear the modification and creation times for this test.
// nolint:descriptormarshal
if tbl := ev.Desc.GetTable(); tbl != nil {
tbl.ModificationTime = hlc.Timestamp{}
tbl.CreateAsOfTime = hlc.Timestamp{}
}
require.Truef(t, ok, "expected a SchemaDescriptor event, instead got %T", event)
require.EqualValues(t, meta.SnapshotID, ev.SnapshotID, "unexpected snapshot ID")
if ev.DescID == keys.PublicSchemaID && ev.Desc == nil {
Expand Down
144 changes: 72 additions & 72 deletions pkg/sql/catalog/systemschema_test/testdata/bootstrap_system

Large diffs are not rendered by default.

144 changes: 72 additions & 72 deletions pkg/sql/catalog/systemschema_test/testdata/bootstrap_tenant

Large diffs are not rendered by default.

3 changes: 3 additions & 0 deletions pkg/sql/catalog/tabledesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ go_test(
"main_test.go",
"safe_format_test.go",
"structured_test.go",
"table_desc_builder_test.go",
"table_desc_test.go",
"validate_test.go",
"validate_version_gating_test.go",
Expand Down Expand Up @@ -111,6 +112,7 @@ go_test(
"//pkg/sql/catalog/funcdesc",
"//pkg/sql/catalog/internal/validate",
"//pkg/sql/catalog/nstree",
"//pkg/sql/catalog/systemschema",
"//pkg/sql/privilege",
"//pkg/sql/schemachanger/scpb",
"//pkg/sql/sem/catconstants",
Expand All @@ -126,6 +128,7 @@ go_test(
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/randutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_kr_pretty//:pretty",
Expand Down
51 changes: 39 additions & 12 deletions pkg/sql/catalog/tabledesc/table_desc_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ type TableDescriptorBuilder interface {
BuildImmutableTable() catalog.TableDescriptor
BuildExistingMutableTable() *Mutable
BuildCreatedMutableTable() *Mutable
DescriptorWasModified() bool
}

type tableDescriptorBuilder struct {
Expand Down Expand Up @@ -108,14 +109,41 @@ func NewUnsafeImmutable(desc *descpb.TableDescriptor) catalog.TableDescriptor {
return b.BuildImmutableTable()
}

// cloneWithModificationStamp will clone the table descriptor and
// set the modification timestamp on it.
func cloneWithModificationStamp(
desc *descpb.TableDescriptor, mvccTimestamp hlc.Timestamp,
) (newDesc *descpb.TableDescriptor, updated bool) {
newDesc = protoutil.Clone(desc).(*descpb.TableDescriptor)
// Set the ModificationTime field before doing anything else.
// Other changes may depend on it. Note: The post deserialization
// will surface the assertions from here, since we can't return
// errors here.
mustSetModTime, err := descpb.MustSetModificationTime(
desc.ModificationTime, mvccTimestamp, desc.Version, desc.State,
)
// If we updated the modification time, then
// now is a good time to pick up the CreateAs time.
if err == nil && mustSetModTime {
updated = true
newDesc.ModificationTime = mvccTimestamp
maybeSetCreateAsOfTime(newDesc)
}
return newDesc, updated
}

func newBuilder(
desc *descpb.TableDescriptor,
mvccTimestamp hlc.Timestamp,
isUncommittedVersion bool,
changes catalog.PostDeserializationChanges,
) *tableDescriptorBuilder {
newDesc, modificationUpdated := cloneWithModificationStamp(desc, mvccTimestamp)
if modificationUpdated {
changes.Add(catalog.SetModTimeToMVCCTimestamp)
}
return &tableDescriptorBuilder{
original: protoutil.Clone(desc).(*descpb.TableDescriptor),
original: newDesc,
mvccTimestamp: mvccTimestamp,
isUncommittedVersion: isUncommittedVersion,
changes: changes,
Expand All @@ -133,18 +161,15 @@ func (tdb *tableDescriptorBuilder) RunPostDeserializationChanges() (err error) {
defer func() {
err = errors.Wrapf(err, "table %q (%d)", tdb.original.Name, tdb.original.ID)
}()
// Set the ModificationTime field before doing anything else.
// Other changes may depend on it.
mustSetModTime, err := descpb.MustSetModificationTime(
// Validate that the modification timestamp is valid. This
// is now done earlier, but we can't return errors if an
// assertion is hit.
_, err = descpb.MustSetModificationTime(
tdb.original.ModificationTime, tdb.mvccTimestamp, tdb.original.Version, tdb.original.State,
)
if err != nil {
return err
}
if mustSetModTime {
tdb.getOrInitModifiedDesc().ModificationTime = tdb.mvccTimestamp
tdb.changes.Add(catalog.SetModTimeToMVCCTimestamp)
}
c, err := maybeFillInDescriptor(tdb)
if err != nil {
return err
Expand All @@ -153,6 +178,11 @@ func (tdb *tableDescriptorBuilder) RunPostDeserializationChanges() (err error) {
return nil
}

// DescriptorWasModified implements TableDescriptorBuilder
func (tdb *tableDescriptorBuilder) DescriptorWasModified() bool {
return tdb.maybeModified != nil
}

// GetReadOnlyPrivilege implements catprivilege.PrivilegeDescriptorBuilder.
func (tdb *tableDescriptorBuilder) GetReadOnlyPrivilege() *catpb.PrivilegeDescriptor {
p := tdb.getLatestDesc().Privileges
Expand Down Expand Up @@ -415,7 +445,6 @@ func maybeFillInDescriptor(
changes.Add(change)
}
}
set(catalog.SetCreateAsOfTimeUsingModTime, maybeSetCreateAsOfTime(builder))
set(catalog.UpgradedFormatVersion, maybeUpgradeFormatVersion(builder))
set(catalog.UpgradedIndexFormatVersion, maybeUpgradePrimaryIndexFormatVersion(builder))
{
Expand Down Expand Up @@ -1032,8 +1061,7 @@ func maybeFixSecondaryIndexEncodingType(builder *tableDescriptorBuilder) (hasCha
// ModificationTime fields are both unset for the first Version of a
// TableDescriptor and the code relies on the value being set based on the
// MVCC timestamp.
func maybeSetCreateAsOfTime(builder *tableDescriptorBuilder) (hasChanged bool) {
desc := builder.getLatestDesc()
func maybeSetCreateAsOfTime(desc *descpb.TableDescriptor) (hasChanged bool) {
if !desc.CreateAsOfTime.IsEmpty() || desc.Version > 1 || desc.ModificationTime.IsEmpty() {
return false
}
Expand All @@ -1045,7 +1073,6 @@ func maybeSetCreateAsOfTime(builder *tableDescriptorBuilder) (hasChanged bool) {
// The expectation is that this is only set when the version is 2.
// For any version greater than that, this is not accurate but better than
// nothing at all.
desc = builder.getOrInitModifiedDesc()
desc.CreateAsOfTime = desc.ModificationTime
return true
}
Expand Down
35 changes: 35 additions & 0 deletions pkg/sql/catalog/tabledesc/table_desc_builder_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
// Copyright 2024 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 tabledesc_test

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/stretchr/testify/require"
)

// TestDescriptorsAreNotCopied validate that descriptors that have
// survived post de-serialziation don't have to go through it again.
func TestDescriptorsAreNotCopied(t *testing.T) {
// Sanity: Loop over the system tables and confirm that
// PostDeserialization does zero work on updated descriptors.
for _, targetDesc := range systemschema.MakeSystemTables() {
ts := hlc.Timestamp{WallTime: timeutil.Now().Unix()}
b := tabledesc.NewBuilderWithMVCCTimestamp(targetDesc.TableDesc(), ts)
require.NoError(t, b.RunPostDeserializationChanges())
require.Falsef(t, b.DescriptorWasModified(),
"%s descriptor was copied again", targetDesc.GetName())
}
}

0 comments on commit e89a581

Please sign in to comment.