Skip to content

Commit

Permalink
Merge #58519 #60695
Browse files Browse the repository at this point in the history
58519: sql: code generation for missing pg catalog tables r=RichardJCai a=mnovelodou

Previously, programmer had to add missing table manually,
This was inadequate because postgress added a lot of tables and
manual process can lead to coding mistakes or gaps
To address this, diff tool now have the hability to generate the
missing code

Release note: None

Fixes #58001

60695: sql: add catalog.Mutation interface for table mutations r=postamar a=postamar

Previously, catalog.TableDescriptor had a method, GetMutations, which
returned a slice of descpb.DescriptorMutation structs, which in turn
contain descpb.IndexDescriptor and descpb.ColumnDescriptor types. This
is an obstacle to our ongoing effort to virtualize column and index
descriptors.

This commit therefore wraps descpb.DescriptorMutation in an interface
type, catalog.Mutation, and also wraps the remaining mutation descriptor
field types:
    - ConstraintToUpdate,
    - PrimaryKeySwap,
    - ComputedColumnSwap,
    - MaterializedViewRefresh.

Release justification: This commit is safe for this release because it
is a low-risk refactor.

Release note: None

Co-authored-by: MiguelNovelo <miguel.novelo@digitalonus.com>
Co-authored-by: Marius Posta <marius@cockroachlabs.com>
  • Loading branch information
3 people committed Mar 11, 2021
3 parents f2f37a8 + 25aeb98 + 6e790fb commit a5b80ff
Show file tree
Hide file tree
Showing 30 changed files with 1,428 additions and 645 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func AddColumnDropBackfillMutation(desc catalog.TableDescriptor) catalog.TableDe
Direction: descpb.DescriptorMutation_DROP,
Descriptor_: &descpb.DescriptorMutation_Column{Column: MakeColumnDesc(desc.GetNextColumnID() - 1)},
})
return desc
return tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable()
}

// AddNewColumnBackfillMutation adds a mutation to desc to add a column.
Expand All @@ -70,5 +70,5 @@ func AddNewColumnBackfillMutation(desc catalog.TableDescriptor) catalog.TableDes
MutationID: 0,
Rollback: false,
})
return desc
return tabledesc.NewBuilder(desc.TableDesc()).BuildImmutableTable()
}
12 changes: 5 additions & 7 deletions pkg/ccl/changefeedccl/schemafeed/table_event_filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -92,12 +91,11 @@ func hasNewColumnDropBackfillMutation(e TableEvent) (res bool) {
}

func dropColumnMutationExists(desc catalog.TableDescriptor) bool {
for _, m := range desc.GetMutations() {
if m.GetColumn() == nil {
for _, m := range desc.AllMutations() {
if m.AsColumn() == nil {
continue
}
if m.Direction == descpb.DescriptorMutation_DROP &&
m.State == descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY {
if m.Dropped() && m.WriteAndDeleteOnly() {
return true
}
}
Expand All @@ -117,8 +115,8 @@ func newColumnNoBackfill(e TableEvent) (res bool) {
}

func pkChangeMutationExists(desc catalog.TableDescriptor) bool {
for _, m := range desc.GetMutations() {
if m.Direction == descpb.DescriptorMutation_ADD && m.GetPrimaryKeySwap() != nil {
for _, m := range desc.AllMutations() {
if m.Adding() && m.AsPrimaryKeySwap() != nil {
return true
}
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/changefeedccl/schemafeed/table_event_filter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/schemafeed/schematestutils"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/gogo/protobuf/proto"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -75,10 +76,10 @@ func TestTableEventFilter(t *testing.T) {
e: TableEvent{
Before: func() catalog.TableDescriptor {
td := addColBackfill(mkTableDesc(42, 4, ts(4), 1))
col := td.GetMutations()[0].GetColumn()
col := td.TableDesc().Mutations[0].GetColumn()
col.Nullable = true
col.ComputeExpr = proto.String("1")
return td
return tabledesc.NewBuilder(td.TableDesc()).BuildImmutableTable()
}(),
After: mkTableDesc(42, 4, ts(4), 2),
},
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/multiregionccl/regional_by_row_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -419,10 +419,10 @@ USE t;
tableDesc := catalogkv.TestingGetTableDescriptor(
kvDB, keys.SystemSQLCodec, "t", "test",
)
if len(tableDesc.GetMutations()) != 0 {
if len(tableDesc.AllMutations()) != 0 {
return errors.Errorf(
"expected 0 mutations after cancellation, found %d",
len(tableDesc.GetMutations()),
len(tableDesc.AllMutations()),
)
}
if tableDesc.GetPrimaryIndex().NumColumns() != len(testCase.originalPKCols) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/jobs/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -870,7 +870,7 @@ func (r *Registry) isOrphaned(ctx context.Context, payload *jobspb.Payload) (boo
if err != nil {
return err
}
hasAnyMutations := len(td.GetMutations()) != 0 || len(td.GetGCMutations()) != 0
hasAnyMutations := len(td.AllMutations()) != 0 || len(td.GetGCMutations()) != 0
hasDropJob := td.TableDesc().DropJobID != 0
pendingMutations = hasAnyMutations || hasDropJob
return nil
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -610,6 +610,7 @@ go_test(
"@com_github_jackc_pgx//pgtype",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_lib_pq//:pq",
"@com_github_lib_pq//oid",
"@com_github_pmezard_go_difflib//difflib",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_column_type_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,7 @@ ALTER TABLE t.test ALTER COLUMN x TYPE INT;
// Ensure that the add column and column swap mutations are cleaned up.
testutils.SucceedsSoon(t, func() error {
desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "test")
if len(desc.GetMutations()) != 0 {
if len(desc.AllMutations()) != 0 {
return errors.New("expected no mutations on TableDescriptor")
}
return nil
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/alter_table_locality.go
Original file line number Diff line number Diff line change
Expand Up @@ -348,7 +348,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow(
}

// Allow add column mutation to be on the same mutation ID in AlterPrimaryKey.
mutationIdx := len(n.tableDesc.GetMutations()) - 1
mutationIdx := len(n.tableDesc.Mutations) - 1
mutationIdxAllowedInSameTxn = &mutationIdx
newColumnName = &partColName

Expand All @@ -363,7 +363,7 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow(
// The primary_region default helps us also have a material value.
// This can be removed when the default_expr can serialize user defined
// functions.
col := n.tableDesc.GetMutations()[mutationIdx].GetColumn()
col := n.tableDesc.Mutations[mutationIdx].GetColumn()
finalDefaultExpr, err := schemaexpr.SanitizeVarFreeExpr(
params.ctx,
regionalByRowGatewayRegionDefaultExpr(enumOID),
Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -1426,19 +1426,19 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error {
var forwardIndexes []*descpb.IndexDescriptor
var invertedIndexes []*descpb.IndexDescriptor

for _, m := range tableDesc.GetMutations() {
if sc.mutationID != m.MutationID {
for _, m := range tableDesc.AllMutations() {
if sc.mutationID != m.MutationID() {
break
}
idx := m.GetIndex()
if idx == nil || m.Direction == descpb.DescriptorMutation_DROP {
idx := m.AsIndex()
if idx == nil || idx.Dropped() {
continue
}
switch idx.Type {
switch idx.GetType() {
case descpb.IndexDescriptor_FORWARD:
forwardIndexes = append(forwardIndexes, idx)
forwardIndexes = append(forwardIndexes, idx.IndexDesc())
case descpb.IndexDescriptor_INVERTED:
invertedIndexes = append(invertedIndexes, idx)
invertedIndexes = append(invertedIndexes, idx.IndexDesc())
}
}
if len(forwardIndexes) == 0 && len(invertedIndexes) == 0 {
Expand Down
33 changes: 16 additions & 17 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,18 +38,17 @@ import (
)

// MutationFilter is the type of a simple predicate on a mutation.
type MutationFilter func(descpb.DescriptorMutation) bool
type MutationFilter func(catalog.Mutation) bool

// ColumnMutationFilter is a filter that allows mutations that add or drop
// columns.
func ColumnMutationFilter(m descpb.DescriptorMutation) bool {
return m.GetColumn() != nil &&
(m.Direction == descpb.DescriptorMutation_ADD || m.Direction == descpb.DescriptorMutation_DROP)
func ColumnMutationFilter(m catalog.Mutation) bool {
return m.AsColumn() != nil && (m.Adding() || m.Dropped())
}

// IndexMutationFilter is a filter that allows mutations that add indexes.
func IndexMutationFilter(m descpb.DescriptorMutation) bool {
return m.GetIndex() != nil && m.Direction == descpb.DescriptorMutation_ADD
func IndexMutationFilter(m catalog.Mutation) bool {
return m.AsIndex() != nil && m.Adding()
}

// ColumnBackfiller is capable of running a column backfill for all
Expand All @@ -72,14 +71,13 @@ type ColumnBackfiller struct {

// initCols is a helper to populate some column metadata on a ColumnBackfiller.
func (cb *ColumnBackfiller) initCols(desc catalog.TableDescriptor) {
for _, m := range desc.GetMutations() {
for _, m := range desc.AllMutations() {
if ColumnMutationFilter(m) {
desc := *m.GetColumn()
switch m.Direction {
case descpb.DescriptorMutation_ADD:
cb.added = append(cb.added, desc)
case descpb.DescriptorMutation_DROP:
cb.dropped = append(cb.dropped, desc)
col := *m.AsColumn().ColumnDesc()
if m.Adding() {
cb.added = append(cb.added, col)
} else if m.Dropped() {
cb.dropped = append(cb.dropped, col)
}
}
}
Expand Down Expand Up @@ -692,16 +690,17 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) {
// fetched in order to backfill the added indexes.
func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIntSet {
var valNeededForCol util.FastIntSet
mutationID := desc.GetMutations()[0].MutationID
mutations := desc.AllMutations()
mutationID := mutations[0].MutationID()

// Mutations in the same transaction have the same ID. Loop through the
// mutations and collect all index mutations.
for _, m := range desc.GetMutations() {
if m.MutationID != mutationID {
for _, m := range mutations {
if m.MutationID() != mutationID {
break
}
if IndexMutationFilter(m) {
idx := m.GetIndex()
idx := m.AsIndex().IndexDesc()
ib.added = append(ib.added, idx)
for i := range ib.cols {
id := ib.cols[i].ID
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"errors.go",
"table_col_map.go",
"table_col_set.go",
"table_elements.go",
"validate.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog",
Expand Down
Loading

0 comments on commit a5b80ff

Please sign in to comment.