Skip to content

Commit

Permalink
Add LogicalColumnID field to ColumnDescriptor.
Browse files Browse the repository at this point in the history
The LogicalColumnID field mimics the ColumnID field however LogicalColumnID may be swapped
between two columns whereas ColumnID cannot. LogicalColumnID is referenced for virtual tables
(pg_catalog, information_schema) and most notably affects column ordering for SHOW COLUMNS.

This LogicalColumnID field support swapping the order of two columns - currently only used for
ALTER COLUMN TYPE when a shadow column is created and swapped with it's original column.

Does not affect existing behaviour.

Release note: None
  • Loading branch information
RichardJCai committed Apr 13, 2020
1 parent 3967538 commit 7afcfd0
Show file tree
Hide file tree
Showing 9 changed files with 605 additions and 432 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,8 @@ func allocateTableRewrites(
// backup descriptors provided. if skipFKsWithNoMatchingTable is set, FKs whose
// "other" table is missing from the set provided are omitted during the
// upgrade, instead of causing an error to be returned.
// Also assigns LogicalColumnIDs for TableDescriptors that do not have it,
// namely TableDescriptors created before 20.2.
func maybeUpgradeTableDescsInBackupManifests(
ctx context.Context, backupManifests []BackupManifest, skipFKsWithNoMatchingTable bool,
) error {
Expand All @@ -413,6 +415,7 @@ func maybeUpgradeTableDescsInBackupManifests(
if _, err := table.MaybeUpgradeForeignKeyRepresentation(ctx, protoGetter, skipFKsWithNoMatchingTable); err != nil {
return err
}
table.MaybeFillInLogicalColumnID()
// TODO(lucy): Is this necessary?
backupManifest.Descriptors[j] = *sqlbase.WrapDescriptor(table)
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/information_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -387,11 +387,11 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`,
collationName = tree.NewDString(locale)
}
return addRow(
dbNameStr, // table_catalog
scNameStr, // table_schema
tree.NewDString(table.Name), // table_name
tree.NewDString(column.Name), // column_name
tree.NewDInt(tree.DInt(column.ID)), // ordinal_position
dbNameStr, // table_catalog
scNameStr, // table_schema
tree.NewDString(table.Name), // table_name
tree.NewDString(column.Name), // column_name
tree.NewDInt(tree.DInt(column.LogicalColumnID)), // ordinal_position
dStringPtrOrNull(column.DefaultExpr), // column_default
yesOrNoDatum(column.Nullable), // is_nullable
tree.NewDString(column.Type.InformationSchemaName()), // data_type
Expand Down
76 changes: 76 additions & 0 deletions pkg/sql/logical_column_id_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql

import (
"context"
"testing"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

// TestAddLogicalColumnID writes a TableDescriptor to disk without
// LogicalColumnIDs set and reads it back expecting the LogicalColumnID
// to be updated on read and equal to ColumnID.
func TestAddLogicalColumnID(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
params, _ := tests.CreateTestServerParams()
s, sqlDB, kvDB := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)

tests := []struct {
name string
query string
}{
{"t1", "CREATE TABLE t1 (x INT);"},
{"t2", "CREATE TABLE t2 (x INT, y INT);"},
}
for i := range tests {
if _, err := sqlDB.Exec(tests[i].query); err != nil {
t.Fatal(err)
}

removeLogicalColumnIDs := func(tDesc *sqlbase.TableDescriptor) {
for i := range tDesc.Columns {
tDesc.Columns[i].LogicalColumnID = 0
}
}

desc := sqlbase.GetTableDescriptor(kvDB, "defaultdb", tests[i].name)

err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
b := txn.NewBatch()
removeLogicalColumnIDs(desc)
if err := writeDescToBatch(ctx, false, s.ClusterSettings(), b, desc.ID, desc); err != nil {
return err
}
return txn.Run(ctx, b)
})

if err != nil {
t.Fatal(err)
}

desc = sqlbase.GetTableDescriptor(kvDB, "defaultdb", desc.Name)

for i := range desc.Columns {
if desc.Columns[i].LogicalColumnID != desc.Columns[i].ID {
t.Fatalf("Expected LogicalColumnID to be %d, got %d.",
desc.Columns[i].ID, desc.Columns[i].LogicalColumnID)
}
}
}
}
42 changes: 20 additions & 22 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -391,9 +391,7 @@ CREATE TABLE pg_catalog.pg_attrdef (
h := makeOidHasher()
return forEachTableDesc(ctx, p, dbContext, virtualMany,
func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error {
colNum := 0
return forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error {
colNum++
if column.DefaultExpr == nil {
// pg_attrdef only expects rows for columns with default values.
return nil
Expand All @@ -408,11 +406,11 @@ CREATE TABLE pg_catalog.pg_attrdef (
defSrc = tree.NewDString(ctx.String())
}
return addRow(
h.ColumnOid(table.ID, column.ID), // oid
defaultOid(table.ID), // adrelid
tree.NewDInt(tree.DInt(colNum)), // adnum
defSrc, // adbin
defSrc, // adsrc
h.ColumnOid(table.ID, column.ID), // oid
defaultOid(table.ID), // adrelid
tree.NewDInt(tree.DInt(column.LogicalColumnID)), // adnum
defSrc, // adbin
defSrc, // adsrc
)
})
})
Expand Down Expand Up @@ -450,7 +448,7 @@ CREATE TABLE pg_catalog.pg_attribute (
h := makeOidHasher()
return forEachTableDesc(ctx, p, dbContext, virtualMany, func(db *sqlbase.DatabaseDescriptor, scName string, table *sqlbase.TableDescriptor) error {
// addColumn adds adds either a table or a index column to the pg_attribute table.
addColumn := func(column *sqlbase.ColumnDescriptor, attRelID tree.Datum, colID sqlbase.ColumnID) error {
addColumn := func(column *sqlbase.ColumnDescriptor, attRelID tree.Datum, logicalColID sqlbase.ColumnID) error {
colTyp := &column.Type
attTypMod := int32(-1)
if width := colTyp.Width(); width != 0 {
Expand All @@ -470,18 +468,18 @@ CREATE TABLE pg_catalog.pg_attribute (
}
}
return addRow(
attRelID, // attrelid
tree.NewDName(column.Name), // attname
typOid(colTyp), // atttypid
zeroVal, // attstattarget
typLen(colTyp), // attlen
tree.NewDInt(tree.DInt(colID)), // attnum
zeroVal, // attndims
negOneVal, // attcacheoff
tree.NewDInt(tree.DInt(attTypMod)), // atttypmod
tree.DNull, // attbyval (see pg_type.typbyval)
tree.DNull, // attstorage
tree.DNull, // attalign
attRelID, // attrelid
tree.NewDName(column.Name), // attname
typOid(colTyp), // atttypid
zeroVal, // attstattarget
typLen(colTyp), // attlen
tree.NewDInt(tree.DInt(logicalColID)), // attnum
zeroVal, // attndims
negOneVal, // attcacheoff
tree.NewDInt(tree.DInt(attTypMod)), // atttypmod
tree.DNull, // attbyval (see pg_type.typbyval)
tree.DNull, // attstorage
tree.DNull, // attalign
tree.MakeDBool(tree.DBool(!column.Nullable)), // attnotnull
tree.MakeDBool(tree.DBool(column.DefaultExpr != nil)), // atthasdef
tree.DBoolFalse, // attisdropped
Expand All @@ -497,7 +495,7 @@ CREATE TABLE pg_catalog.pg_attribute (
// Columns for table.
if err := forEachColumnInTable(table, func(column *sqlbase.ColumnDescriptor) error {
tableID := defaultOid(table.ID)
return addColumn(column, tableID, column.ID)
return addColumn(column, tableID, column.LogicalColumnID)
}); err != nil {
return err
}
Expand All @@ -507,7 +505,7 @@ CREATE TABLE pg_catalog.pg_attribute (
return forEachColumnInIndex(table, index,
func(column *sqlbase.ColumnDescriptor) error {
idxID := h.IndexOid(table.ID, index.ID)
return addColumn(column, idxID, column.ID)
return addColumn(column, idxID, column.LogicalColumnID)
},
)
})
Expand Down
31 changes: 30 additions & 1 deletion pkg/sql/sqlbase/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -387,6 +387,9 @@ func GetTableDescFromIDWithFKsChanged(
if err != nil {
return nil, false, err
}

table.MaybeFillInLogicalColumnID()

return table, changed, err
}

Expand Down Expand Up @@ -889,11 +892,16 @@ func (desc *TableDescriptor) MaybeFillInDescriptor(
) error {
desc.maybeUpgradeFormatVersion()
desc.Privileges.MaybeFixPrivileges(desc.ID)

var err error
if protoGetter != nil {
if _, err := desc.MaybeUpgradeForeignKeyRepresentation(ctx, protoGetter, false /* skipFKsWithNoMatchingTable*/); err != nil {
if _, err = desc.MaybeUpgradeForeignKeyRepresentation(ctx, protoGetter, false /* skipFKsWithNoMatchingTable*/); err != nil {
return err
}
}

desc.MaybeFillInLogicalColumnID()

return nil
}

Expand Down Expand Up @@ -1198,6 +1206,27 @@ func (desc *MutableTableDescriptor) MaybeFillColumnID(
}
columnNames[c.Name] = columnID
c.ID = columnID
c.LogicalColumnID = columnID
}

// MaybeFillInLogicalColumnID assigns a LogicalColumnID to a column
// if the ColumnID is set and the LogicalColumnID is not
// it assigns the ColumnID as the LogicalColumnID.
func (desc *TableDescriptor) MaybeFillInLogicalColumnID() {
for i := range desc.Columns {
col := &desc.Columns[i]
if col.ID != 0 && col.LogicalColumnID == 0 {
col.LogicalColumnID = col.ID
}
}

for _, m := range desc.Mutations {
if col := m.GetColumn(); col != nil {
if col.ID != 0 && col.LogicalColumnID == 0 {
col.LogicalColumnID = col.ID
}
}
}
}

// AllocateIDs allocates column, family, and index ids for any column, family,
Expand Down
Loading

0 comments on commit 7afcfd0

Please sign in to comment.