Skip to content

Commit

Permalink
Merge #66599 #66664
Browse files Browse the repository at this point in the history
66599: sql: enable adding columns which are not written to the current primary index r=postamar a=postamar

    row,schemachanger: use StoreColumnIDs/Names in primary index
    
    In a recent commit, the StoreColumnIDs and StoreColumnNames slices in
    primary indexes were populated when previously they had simply been
    empty. We simply assumed that all non-virtual columns in a table would
    be stored in the primary index: primary key columns in the key, the rest
    in the value.
    
    This commit breaks that assumption by using the StoreColumnIDs slice to
    determine what goes into the primary index. This makes it possible for
    the new schema changer to add columns safely, preventing unwanted writes
    to the old primary index while the schema change is underway.
    
    Fixes #59149.
    
    Release note: None


    sql,tabledesc: add new IndexDescriptorVersion for primary indexes
    
    Previously, the IndexDescriptorVersion type was only used to describe
    the encoding of secondary indexes. This commit adds a new value for
    use in primary indexes, PrimaryIndexWithStoredColumnsVersion, to signify
    that the StoredColumnIDs and StoredColumnNames slices are populated
    correctly.
    
    Previously, these slices did not need to be populated at all. This is
    because the set of columns comprising the primary index of a table is
    assumed to be all non-virtual columns of that table. Our upcoming work on
    the new schema changer will require us to violate that assumption
    however. This commit is in preparation of that change.
    
    In our effort to make meaningful the concept of stored columns in
    primary indexes, this commit also changes the contents of the
    information_schema.statistics table. As a result, SHOW INDEXES and SHOW
    COLUMNS behave the same way regardless of whether an index is primary or
    secondary.
    
    Release note (sql change): The contents of the statistics table in the
    information schema have changed, therefore so have the results of SHOW
    INDEX and SHOW COLUMNS. A column which is not in the primary key will
    now be listed as belonging to the primary index as a stored column.
    Previously, it was simply not listed as belonging to the primary index.

66664: sql: First round of cleanup of schemachange/random-load r=ajwerner,otan a=ajstorm

This issue addresses several issues uncovered in running the randomized
schema changer. Specifically:

- Makes several errors pgcodes, so that they can be properly added to
  the expected errors list in the randomized schema changer.
- Detects cases where the region column (crdb_region) is used multiple
  times in an index definition.
- Allows for column type changes, which must have the experimental flag
  enable_experimental_alter_column_type_general flag set.

It also disables the testing of setColumnType (tracked with #66662) as
well as making a column nullable/non-nullable due to a timing hole
(tracked with #66663).

Release note: None.

Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: Adam Storm <[email protected]>
  • Loading branch information
3 people committed Jun 22, 2021
3 parents ecbd962 + 3dd87a4 + 46baa75 commit 2cafdfb
Show file tree
Hide file tree
Showing 86 changed files with 1,602 additions and 986 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -521,6 +521,14 @@ created_idx c false
created_idx pk true
primary partition_by true
primary pk false
primary pk2 false
primary a false
primary b false
primary c false
primary d false
primary j false
primary u false
primary e false
t_a_idx partition_by true
t_a_idx a false
t_a_idx pk true
Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/regional_by_row
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,10 @@ ORDER BY index_name, seq_in_index
----
primary crdb_region true
primary pk false
primary pk2 false
primary a false
primary b false
primary j false
regional_by_row_table_a_idx crdb_region true
regional_by_row_table_a_idx a false
regional_by_row_table_a_idx pk true
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/multiregionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,16 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/descpb",
"//pkg/sql/execinfra",
"//pkg/sql/sqltestutils",
"//pkg/sql/tests",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/leaktest",
Expand Down
133 changes: 133 additions & 0 deletions pkg/ccl/multiregionccl/regional_by_row_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,23 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils"
"github.com/cockroachdb/cockroach/pkg/ccl/testutilsccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -887,3 +891,132 @@ USE t;
}
}
}

// TestIndexDescriptorUpdateForImplicitColumns checks that the column ID slices
// in the indexes of a table descriptor undergoing partitioning changes
// involving implicit columns are correctly updated.
func TestIndexDescriptorUpdateForImplicitColumns(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
defer utilccl.TestingEnableEnterprise()()

c, sqlDB, cleanup := multiregionccltestutils.TestingCreateMultiRegionCluster(
t, 3 /* numServers */, base.TestingKnobs{},
)
defer cleanup()

tdb := sqlutils.MakeSQLRunner(sqlDB)
tdb.Exec(t, `CREATE DATABASE test PRIMARY REGION "us-east1" REGIONS "us-east2"`)

fetchIndexes := func(tableName string) []catalog.Index {
kvDB := c.Servers[0].DB()
desc := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "test", tableName)
return desc.NonDropIndexes()
}

t.Run("primary index", func(t *testing.T) {
tdb.Exec(t, `CREATE TABLE test.t1 (
a INT PRIMARY KEY,
b test.public.crdb_internal_region NOT NULL
) LOCALITY GLOBAL`)
indexes := fetchIndexes("t1")
require.Len(t, indexes, 1)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t1 SET LOCALITY REGIONAL BY ROW AS b`)
indexes = fetchIndexes("t1")
require.Len(t, indexes, 1)

require.EqualValues(t, []descpb.ColumnID{1, 2}, indexes[0].CollectKeyColumnIDs().Ordered())
require.Empty(t, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t1 SET LOCALITY GLOBAL`)
indexes = fetchIndexes("t1")
require.Len(t, indexes, 1)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())
})

t.Run("secondary index", func(t *testing.T) {
tdb.Exec(t, `CREATE TABLE test.t2 (
a INT PRIMARY KEY,
b test.public.crdb_internal_region NOT NULL,
c INT NOT NULL,
d INT NOT NULL,
INDEX sec (c) STORING (d)
) LOCALITY GLOBAL`)
indexes := fetchIndexes("t2")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2, 3, 4}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{3}, indexes[1].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{1}, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{4}, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t2 SET LOCALITY REGIONAL BY ROW AS b`)
indexes = fetchIndexes("t2")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1, 2}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{3, 4}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{2, 3}, indexes[1].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{1}, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{4}, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t2 SET LOCALITY GLOBAL`)
indexes = fetchIndexes("t2")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2, 3, 4}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{3}, indexes[1].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{1}, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{4}, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())
})

t.Run("secondary index key suffix", func(t *testing.T) {
tdb.Exec(t, `CREATE TABLE test.t3 (
a test.public.crdb_internal_region PRIMARY KEY,
b INT NOT NULL,
INDEX sec (b)
) LOCALITY GLOBAL`)
indexes := fetchIndexes("t3")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{2}, indexes[1].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{1}, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.Empty(t, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t3 SET LOCALITY REGIONAL BY ROW AS a`)
indexes = fetchIndexes("t3")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{1, 2}, indexes[1].CollectKeyColumnIDs().Ordered())
require.Empty(t, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.Empty(t, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())

tdb.Exec(t, `ALTER TABLE test.t3 SET LOCALITY GLOBAL`)
indexes = fetchIndexes("t3")
require.Len(t, indexes, 2)

require.EqualValues(t, []descpb.ColumnID{1}, indexes[0].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{2}, indexes[0].CollectPrimaryStoredColumnIDs().Ordered())

require.EqualValues(t, []descpb.ColumnID{2}, indexes[1].CollectKeyColumnIDs().Ordered())
require.EqualValues(t, []descpb.ColumnID{1}, indexes[1].CollectKeySuffixColumnIDs().Ordered())
require.Empty(t, indexes[1].CollectSecondaryStoredColumnIDs().Ordered())
})
}
4 changes: 2 additions & 2 deletions pkg/cli/sql_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ SET
{`parentID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`},
{`parentSchemaID`, `INT8`, `false`, `NULL`, ``, `{primary}`, `false`},
{`name`, `STRING`, `false`, `NULL`, ``, `{primary}`, `false`},
{`id`, `INT8`, `true`, `NULL`, ``, `{}`, `false`},
{`id`, `INT8`, `true`, `NULL`, ``, `{primary}`, `false`},
}
if !reflect.DeepEqual(expectedRows, rows) {
t.Fatalf("expected:\n%v\ngot:\n%v", expectedRows, rows)
Expand All @@ -172,7 +172,7 @@ SET
parentID | INT8 | false | NULL | | {primary} | false
parentSchemaID | INT8 | false | NULL | | {primary} | false
name | STRING | false | NULL | | {primary} | false
id | INT8 | true | NULL | | {} | false
id | INT8 | true | NULL | | {primary} | false
(4 rows)
`

Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/schemachange_random_load.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ func runSchemaChangeRandomLoad(ctx context.Context, t *test, c Cluster, maxOps,
// saveArtifacts saves important test artifacts in the artifacts directory.
func saveArtifacts(ctx context.Context, t *test, c Cluster, storeDirectory string) {
db := c.Conn(ctx, 1)
defer db.Close()

// Save a backup file called schemachange to the store directory.
_, err := db.Exec("BACKUP DATABASE schemachange to 'nodelocal://1/schemachange'")
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/kvcoord/testdata/savepoints
Original file line number Diff line number Diff line change
Expand Up @@ -385,7 +385,7 @@ injected error

rollback x
----
(*telemetrykeys.withTelemetry) unimplemented: cannot rollback to savepoint after error
(*pgerror.withCandidateCode) unimplemented: cannot rollback to savepoint after error

subtest end

Expand Down
4 changes: 4 additions & 0 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -733,6 +733,10 @@ func TestAdminAPITableDetails(t *testing.T) {

// Verify indexes.
expIndexes := []serverpb.TableDetailsResponse_Index{
{Name: "primary", Column: "string_default", Direction: "N/A", Unique: true, Seq: 5, Storing: true},
{Name: "primary", Column: "default2", Direction: "N/A", Unique: true, Seq: 4, Storing: true},
{Name: "primary", Column: "nulls_not_allowed", Direction: "N/A", Unique: true, Seq: 3, Storing: true},
{Name: "primary", Column: "nulls_allowed", Direction: "N/A", Unique: true, Seq: 2, Storing: true},
{Name: "primary", Column: "rowid", Direction: "ASC", Unique: true, Seq: 1},
{Name: "descidx", Column: "rowid", Direction: "ASC", Unique: false, Seq: 2, Implicit: true},
{Name: "descidx", Column: "default2", Direction: "DESC", Unique: false, Seq: 1},
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/add_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,14 @@ func (p *planner) addColumnImpl(
col.ComputeExpr = &serializedExpr
}

if !col.Virtual {
// Add non-virtual column name and ID to primary index.
primaryIndex := n.tableDesc.GetPrimaryIndex().IndexDescDeepCopy()
primaryIndex.StoreColumnNames = append(primaryIndex.StoreColumnNames, col.Name)
primaryIndex.StoreColumnIDs = append(primaryIndex.StoreColumnIDs, col.ID)
n.tableDesc.SetPrimaryIndex(primaryIndex)
}

// Zone configuration logic is only required for REGIONAL BY ROW tables
// with newly created indexes.
if n.tableDesc.IsLocalityRegionalByRow() && idx != nil {
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,6 +384,13 @@ func alterColumnTypeGeneral(
}

tableDesc.AddColumnMutation(&newCol, descpb.DescriptorMutation_ADD)
if !newCol.Virtual {
// Add non-virtual column name and ID to primary index.
primaryIndex := tableDesc.GetPrimaryIndex().IndexDescDeepCopy()
primaryIndex.StoreColumnNames = append(primaryIndex.StoreColumnNames, newCol.Name)
primaryIndex.StoreColumnIDs = append(primaryIndex.StoreColumnIDs, newCol.ID)
tableDesc.SetPrimaryIndex(primaryIndex)
}

if err := tableDesc.AllocateIDs(ctx); err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func (n *alterIndexNode) startExec(params runParams) error {
"cannot ALTER INDEX and change the partitioning to contain implicit columns",
)
}
isIndexAltered := tabledesc.UpdateIndexPartitioning(&alteredIndexDesc, newImplicitCols, newPartitioning)
isIndexAltered := tabledesc.UpdateIndexPartitioning(&alteredIndexDesc, n.index.Primary(), newImplicitCols, newPartitioning)
if isIndexAltered {
oldPartitioning := n.index.GetPartitioning().DeepCopy()
if n.index.Primary() {
Expand Down
53 changes: 39 additions & 14 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,6 +240,26 @@ func (p *planner) AlterPrimaryKey(
if err := newPrimaryIndexDesc.FillColumns(alterPKNode.Columns); err != nil {
return err
}

{
// Add all deletable non-virtual non-pk columns to new primary index.
names := make(map[string]struct{}, len(newPrimaryIndexDesc.KeyColumnNames))
for _, name := range newPrimaryIndexDesc.KeyColumnNames {
names[name] = struct{}{}
}
deletable := tableDesc.DeletableColumns()
newPrimaryIndexDesc.StoreColumnNames = make([]string, 0, len(deletable))
for _, col := range deletable {
if _, found := names[col.GetName()]; found || col.IsVirtual() {
continue
}
newPrimaryIndexDesc.StoreColumnNames = append(newPrimaryIndexDesc.StoreColumnNames, col.GetName())
}
if len(newPrimaryIndexDesc.StoreColumnNames) == 0 {
newPrimaryIndexDesc.StoreColumnNames = nil
}
}

if err := tableDesc.AddIndexMutation(newPrimaryIndexDesc, descpb.DescriptorMutation_ADD); err != nil {
return err
}
Expand Down Expand Up @@ -349,25 +369,28 @@ func (p *planner) AlterPrimaryKey(
if err != nil {
return err
}
tabledesc.UpdateIndexPartitioning(newPrimaryIndexDesc, newImplicitCols, newPartitioning)
tabledesc.UpdateIndexPartitioning(newPrimaryIndexDesc, true /* isIndexPrimary */, newImplicitCols, newPartitioning)
}

// Ensure that the new primary index stores all columns in the table.
tabledesc.PopulateAllStoreColumns(newPrimaryIndexDesc, tableDesc)
newPrimaryIndexDesc.KeySuffixColumnIDs = nil

// Create a new index that indexes everything the old primary index
// does, but doesn't store anything.
if shouldCopyPrimaryKey(tableDesc, newPrimaryIndexDesc, alterPrimaryKeyLocalitySwap) {
oldPrimaryIndexCopy := tableDesc.GetPrimaryIndex().IndexDescDeepCopy()
// Clear the name of the index so that it gets generated by AllocateIDs.
oldPrimaryIndexCopy.Name = ""
oldPrimaryIndexCopy.StoreColumnIDs = nil
oldPrimaryIndexCopy.StoreColumnNames = nil
newUniqueIdx := tableDesc.GetPrimaryIndex().IndexDescDeepCopy()
// Clear the following fields so that they get generated by AllocateIDs.
newUniqueIdx.ID = 0
newUniqueIdx.Name = ""
newUniqueIdx.StoreColumnIDs = nil
newUniqueIdx.StoreColumnNames = nil
newUniqueIdx.KeySuffixColumnIDs = nil
newUniqueIdx.CompositeColumnIDs = nil
newUniqueIdx.KeyColumnIDs = nil
// Make the copy of the old primary index not-interleaved. This decision
// can be revisited based on user experience.
oldPrimaryIndexCopy.Interleave = descpb.InterleaveDescriptor{}
if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &oldPrimaryIndexCopy, newPrimaryIndexDesc); err != nil {
newUniqueIdx.Interleave = descpb.InterleaveDescriptor{}
// Set correct version and encoding type.
newUniqueIdx.Version = descpb.StrictIndexColumnIDGuaranteesVersion
newUniqueIdx.EncodingType = descpb.SecondaryIndexEncoding
if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newUniqueIdx, newPrimaryIndexDesc); err != nil {
return err
}
}
Expand Down Expand Up @@ -441,7 +464,7 @@ func (p *planner) AlterPrimaryKey(

// Drop any PARTITION ALL BY clause.
if dropPartitionAllBy {
tabledesc.UpdateIndexPartitioning(&newIndex, nil /* newImplicitCols */, descpb.PartitioningDescriptor{})
tabledesc.UpdateIndexPartitioning(&newIndex, idx.Primary(), nil /* newImplicitCols */, descpb.PartitioningDescriptor{})
}

// Create partitioning if we are newly adding a PARTITION BY ALL statement.
Expand All @@ -459,10 +482,12 @@ func (p *planner) AlterPrimaryKey(
if err != nil {
return err
}
tabledesc.UpdateIndexPartitioning(&newIndex, newImplicitCols, newPartitioning)
tabledesc.UpdateIndexPartitioning(&newIndex, idx.Primary(), newImplicitCols, newPartitioning)
}

newIndex.Name = tabledesc.GenerateUniqueName(basename, nameExists)
newIndex.Version = descpb.StrictIndexColumnIDGuaranteesVersion
newIndex.EncodingType = descpb.SecondaryIndexEncoding
if err := addIndexMutationWithSpecificPrimaryKey(ctx, tableDesc, &newIndex, newPrimaryIndexDesc); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -863,7 +863,7 @@ func (n *alterTableNode) startExec(params runParams) error {
"cannot ALTER TABLE and change the partitioning to contain implicit columns",
)
}
isIndexAltered := tabledesc.UpdateIndexPartitioning(&newPrimaryIndexDesc, newImplicitCols, newPartitioning)
isIndexAltered := tabledesc.UpdateIndexPartitioning(&newPrimaryIndexDesc, true /* isIndexPrimary */, newImplicitCols, newPartitioning)
if isIndexAltered {
n.tableDesc.SetPrimaryIndex(newPrimaryIndexDesc)
descriptorChanged = true
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/catalog/descpb/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,14 @@ const (
// each column ID in the ColumnIDs, StoreColumnIDs and KeySuffixColumnIDs
// slices are unique within each slice, and the slices form disjoint sets.
StrictIndexColumnIDGuaranteesVersion
// PrimaryIndexWithStoredColumnsVersion corresponds to the encoding of
// primary indexes that is identical to the unspecified scheme previously in
// use (the IndexDescriptorVersion type was originally only used for
// secondary indexes) but with the guarantee that the StoreColumnIDs and
// StoreColumnNames slices are explicitly populated and maintained. Previously
// these were implicitly derived based on the set of non-virtual columns in
// the table.
PrimaryIndexWithStoredColumnsVersion
)

// ColumnID is a custom type for ColumnDescriptor IDs.
Expand Down
Loading

0 comments on commit 2cafdfb

Please sign in to comment.