Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
135918: schemachanger: remove subzone fallbacks r=annrpom a=annrpom

See individual commits for details.

Fixes: cockroachdb#130900

Release note: None

Co-authored-by: Annie Pompa <[email protected]>
  • Loading branch information
craig[bot] and annrpom committed Dec 6, 2024
2 parents a841259 + 9ebf7f1 commit 21d1123
Show file tree
Hide file tree
Showing 27 changed files with 371 additions and 168 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1106,9 +1106,6 @@ CREATE TABLE tbl2 (
statement ok
SET override_multi_region_zone_config = true;

statement ok
SET use_declarative_schema_changer = off;

statement ok
ALTER INDEX tbl2@tbl2_i_idx CONFIGURE ZONE USING num_replicas=10;

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/zone
Original file line number Diff line number Diff line change
Expand Up @@ -1354,7 +1354,7 @@ CREATE TABLE person (
)
);

skipif config local-mixed-24.3
skipif config local-mixed-24.3 local-legacy-schema-changer
statement ok
SET use_declarative_schema_changer = unsafe_always;

Expand Down
9 changes: 5 additions & 4 deletions pkg/ccl/partitionccl/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,10 +84,11 @@ func TestDropIndexWithZoneConfigCCL(t *testing.T) {

// Set zone configs on the primary index, secondary index, and one partition
// of the secondary index.
ttlYaml := "gc: {ttlseconds: 1}"
sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@kv_pkey", "")
sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@i", ttlYaml)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p2 OF INDEX t.kv@i", ttlYaml)
ttl := "gc.ttlseconds = 1"
sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@kv_pkey",
fmt.Sprintf("num_replicas = %d", *s.DefaultZoneConfig().NumReplicas))
sqlutils.SetZoneConfig(t, sqlDB, "INDEX t.kv@i", ttl)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p2 OF INDEX t.kv@i", ttl)

// Drop the index and verify that the zone config for the secondary index and
// its partition are removed but the zone config for the primary index
Expand Down
24 changes: 12 additions & 12 deletions pkg/ccl/partitionccl/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,8 +64,8 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
PARTITION p1 VALUES IN (DEFAULT)
)`)

yamlDefault := fmt.Sprintf("gc: {ttlseconds: %d}", s.DefaultZoneConfig().GC.TTLSeconds)
yamlOverride := "gc: {ttlseconds: 42}"
gcDefault := fmt.Sprintf("gc.ttlseconds = %d", s.DefaultZoneConfig().GC.TTLSeconds)
gcOverride := "gc.ttlseconds = 42"
zoneOverride := s.DefaultZoneConfig()
zoneOverride.GC = &zonepb.GCPolicy{TTLSeconds: 42}
partialZoneOverride := *zonepb.NewZoneConfig()
Expand Down Expand Up @@ -140,7 +140,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure a database zone config applies to that database, its tables, and its
// tables' indices and partitions.
sqlutils.SetZoneConfig(t, sqlDB, "DATABASE d", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "DATABASE d", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow, partialDbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", dbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", dbRow)
Expand All @@ -150,7 +150,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure a table zone config applies to that table and its indices and
// partitions, but no other zones.
sqlutils.SetZoneConfig(t, sqlDB, "TABLE d.t", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "TABLE d.t", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow, partialDbRow, partialTableRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", dbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", tableRow)
Expand All @@ -160,7 +160,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure an index zone config applies to that index and its partitions, but
// no other zones.
sqlutils.SetZoneConfig(t, sqlDB, "INDEX d.t@t_pkey", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "INDEX d.t@t_pkey", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow, partialDbRow, partialTableRow, partialPrimaryRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", dbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", tableRow)
Expand All @@ -170,7 +170,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure a partition zone config applies to that partition, but no other
// zones.
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE d.t", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE d.t", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow, partialDbRow, partialTableRow, partialPrimaryRow, partialP0Row)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", dbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", tableRow)
Expand All @@ -180,7 +180,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure updating the default zone propagates to zones without an override,
// but not to those with overrides.
sqlutils.SetZoneConfig(t, sqlDB, "RANGE default", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "RANGE default", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultOverrideRow, partialDbRow, partialTableRow, partialPrimaryRow, partialP0Row)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", dbRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", tableRow)
Expand Down Expand Up @@ -223,7 +223,7 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {

// Ensure updating the default zone config applies to zones that have had
// overrides added and removed.
sqlutils.SetZoneConfig(t, sqlDB, "RANGE default", yamlDefault)
sqlutils.SetZoneConfig(t, sqlDB, "RANGE default", gcDefault)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "RANGE default", defaultRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "DATABASE d", defaultRow)
Expand All @@ -233,19 +233,19 @@ func TestValidIndexPartitionSetShowZones(t *testing.T) {
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "PARTITION p1 OF TABLE d.t", defaultRow)

// Ensure subzones can be created even when no table zone exists.
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE d.t", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p1 OF TABLE d.t", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE d.t", gcOverride)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p1 OF TABLE d.t", gcOverride)
sqlutils.VerifyAllZoneConfigs(t, sqlDB, defaultRow, partialP0Row, partialP1Row)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "TABLE d.t", defaultRow)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "PARTITION p0 OF TABLE d.t", p0Row)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "PARTITION p1 OF TABLE d.t", p1Row)

// Ensure the shorthand index syntax works.
sqlutils.SetZoneConfig(t, sqlDB, `INDEX "t_pkey"`, yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, `INDEX "t_pkey"`, gcOverride)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, `INDEX "t_pkey"`, primaryRow)

// Ensure the session database is respected.
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE t", yamlOverride)
sqlutils.SetZoneConfig(t, sqlDB, "PARTITION p0 OF TABLE t", gcOverride)
sqlutils.VerifyZoneConfigForTarget(t, sqlDB, "PARTITION p0 OF TABLE t", p0Row)
}

Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/testutilsccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"//pkg/sql",
"//pkg/sql/execinfra",
"//pkg/sql/sqltestutils",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/util",
Expand Down
17 changes: 14 additions & 3 deletions pkg/ccl/testutilsccl/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"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/util"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -115,9 +117,18 @@ USE t;
// Insert some rows so we can interrupt inspect state during backfill.
require.NoError(t, sqltestutils.BulkInsertIntoTable(sqlDB, maxValue))

runCheck = true
_, err := sqlDB.Exec(tc.AlterQuery)
require.NoError(t, err)
testutils.RunTrueAndFalse(t, "uses-declarative-for-alter-table",
func(t *testing.T, useDeclarativeSchemaChangerForAlter bool) {
if useDeclarativeSchemaChangerForAlter {
skip.WithIssue(t, 136846)
} else {
_, err := sqlDB.Exec("SET CLUSTER SETTING sql.schema.force_declarative_statements = '!ALTER TABLE';")
require.NoError(t, err)
}
runCheck = true
_, err := sqlDB.Exec(tc.AlterQuery)
require.NoError(t, err)
})
})
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_decommission_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ func TestDecommission(t *testing.T) {
// off from the five-fold replicated system ranges.
requireOnlyAtomicChanges(t, runner, tc.LookupRangeOrFatal(t, k).RangeID, triplicated, ts)

sqlutils.SetZoneConfig(t, runner, "RANGE default", "num_replicas: 1")
sqlutils.SetZoneConfig(t, runner, "RANGE default", "num_replicas = 1")

const single = 1

Expand Down
20 changes: 0 additions & 20 deletions pkg/sql/drop_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -345,26 +345,6 @@ func (p *planner) dropIndexByName(
)
}

// Check if requires CCL binary for eventual zone config removal.
_, zone, _, err := GetZoneConfigInTxn(
ctx, p.txn, p.Descriptors(), tableDesc.ID, nil /* index */, "", false,
)
if err != nil {
return err
}

for _, s := range zone.Subzones {
if s.IndexID != uint32(idx.GetID()) {
_, err = GenerateSubzoneSpans(p.ExecCfg().Codec, tableDesc, zone.Subzones)
if sqlerrors.IsCCLRequiredError(err) {
return sqlerrors.NewCCLRequiredError(fmt.Errorf("schema change requires a CCL binary "+
"because table %q has at least one remaining index or partition with a zone config",
tableDesc.Name))
}
break
}
}

// Remove all foreign key references and backreferences from the index.
// TODO (lucy): This is incorrect for two reasons: The first is that FKs won't
// be restored if the DROP INDEX is rolled back, and the second is that
Expand Down
77 changes: 76 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/zone_config
Original file line number Diff line number Diff line change
Expand Up @@ -417,7 +417,7 @@ subtest end

subtest txn_zone_config

skipif config local-mixed-24.3
skipif config local-mixed-24.3 local-legacy-schema-changer
statement ok
SET use_declarative_schema_changer = unsafe_always;

Expand Down Expand Up @@ -450,3 +450,78 @@ statement ok
RESET use_declarative_schema_changer;

subtest end

# Ensure that temp indexes created during a backfill can properly be
# zone-configured.
subtest zc_backfill

statement ok
CREATE TABLE t(i INT, j INT NOT NULL, INDEX idx (j))

skipif config local-mixed-24.3 local-legacy-schema-changer
statement ok
SET use_declarative_schema_changer = unsafe_always;

statement ok
BEGIN;
ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j);
ALTER INDEX t@t_pkey CONFIGURE ZONE USING num_replicas = 11;
COMMIT;

query I colnames
WITH subzones AS (
SELECT
json_array_elements(
crdb_internal.pb_to_json('cockroach.config.zonepb.ZoneConfig', config) -> 'subzones'
) AS config
FROM system.zones
WHERE id = 't'::REGCLASS::OID
),
subzone_configs AS (
SELECT
(config -> 'config' ->> 'numReplicas')::INT AS replicas
FROM subzones
)
SELECT *
FROM subzone_configs
----
replicas
11

skipif config local-mixed-24.3 local-legacy-schema-changer
# Ensure that the subzone config exists on the new primary index. The legacy
# schema changer does not do this; so, we skip those test configs.
query B colnames
WITH subzones AS (
SELECT
json_array_elements(
crdb_internal.pb_to_json('cockroach.config.zonepb.ZoneConfig', config) -> 'subzones'
) AS config
FROM system.zones
WHERE id = 't'::REGCLASS::OID
),
subzone_indexes AS (
SELECT
(config -> 'indexId')::INT AS indexID
FROM subzones
),
primary_index AS (
SELECT
(crdb_internal.pb_to_json(
'cockroach.sql.sqlbase.Descriptor',
descriptor
)->'table'->'primaryIndex'->>'id')::INT AS primaryID
FROM system.descriptor
WHERE id = 't'::regclass::oid
)
SELECT
(primaryID = indexID) AS match_found
FROM primary_index, subzone_indexes;
----
match_found
true

statement ok
RESET use_declarative_schema_changer;

subtest end
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,6 @@ func alterTableAddColumn(
b BuildCtx, tn *tree.TableName, tbl *scpb.Table, stmt tree.Statement, t *tree.AlterTableAddColumn,
) {
d := t.ColumnDef
// We don't support handling zone config related properties for tables, so
// throw an unsupported error.
fallBackIfSubZoneConfigExists(b, t, tbl.TableID)
fallBackIfRegionalByRowTable(b, t, tbl.TableID)

// Check column non-existence.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@ func alterPrimaryKey(
fallBackIfShardedIndexExists(b, t, tbl.TableID)
fallBackIfPartitionedIndexExists(b, t, tbl.TableID)
fallBackIfRegionalByRowTable(b, t.n, tbl.TableID)
fallBackIfSubZoneConfigExists(b, t.n, tbl.TableID)

inflatedChain := getInflatedPrimaryIndexChain(b, tbl.TableID)
if !haveSameIndexColsByKind(b, tbl.TableID, inflatedChain.oldSpec.primary.IndexID,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ func alterTableDropColumn(
stmt tree.Statement,
n *tree.AlterTableDropColumn,
) {
fallBackIfSubZoneConfigExists(b, n, tbl.TableID)
fallBackIfRegionalByRowTable(b, n, tbl.TableID)
checkSafeUpdatesForDropColumn(b)
checkRegionalByRowColumnConflict(b, tbl, n)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,12 +148,6 @@ func astToZoneConfigObject(b BuildCtx, n *tree.SetZoneConfig) (zoneConfigObject,
return nil, scerrors.NotImplementedErrorf(n, "referencing an index without a table "+
"prefix is not supported in the DSC")
}

if !zs.TargetsTable() {
return nil, scerrors.NotImplementedErrorf(n, "zone configurations on system ranges "+
"are not supported in the DSC")
}

// If this is an ALTER ALL PARTITIONS statement, fallback to the legacy schema
// changer.
if zs.TargetsPartition() && zs.StarIndex {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,14 +88,15 @@ func maybeDropIndex(
"use DROP CONSTRAINT ... PRIMARY KEY followed by ADD CONSTRAINT ... PRIMARY KEY in a transaction",
))
}
// TODO (Xiang): Check if requires CCL binary for eventual zone config removal.
_, _, sie := scpb.FindSecondaryIndex(toBeDroppedIndexElms)
if sie == nil {
panic(errors.AssertionFailedf("programming error: cannot find secondary index element."))
}
// We don't support handling zone config related properties for tables, so
// throw an unsupported error.
fallBackIfSubZoneConfigExists(b, nil, sie.TableID)
// TODO(136320): If a region is being added or dropped on this database
// concurrently, we have to block dropping an index for RBR tables. The logic
// for detecting a concurrent ADD/DROP region is not yet in the DSC; falling
// back for now.
fallBackIfRegionalByRowTable(b, nil, sie.TableID)
panicIfSchemaChangeIsDisallowed(b.QueryByID(sie.TableID), n)
// Cannot drop the index if not CASCADE and a unique constraint depends on it.
if n.DropBehavior != tree.DropCascade && sie.IsUnique && !sie.IsCreatedExplicitly {
Expand Down
18 changes: 0 additions & 18 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -862,24 +862,6 @@ func makeSwapIndexSpec(
return in, temp
}

// fallBackIfSubZoneConfigExists determines if the table has a subzone
// config. Normally this logic is used to limit index related operations,
// since dropping indexes will need to remove entries of sub zones from
// the zone config.
func fallBackIfSubZoneConfigExists(b BuildCtx, n tree.NodeFormatter, id catid.DescID) {
{
tableElts := b.QueryByID(id)
if _, _, elem := scpb.FindIndexZoneConfig(tableElts); elem != nil {
panic(scerrors.NotImplementedErrorf(n,
"sub zone configs are not supported"))
}
if _, _, elem := scpb.FindPartitionZoneConfig(tableElts); elem != nil {
panic(scerrors.NotImplementedErrorf(n,
"sub zone configs are not supported"))
}
}
}

// ExtractColumnIDsInExpr extracts column IDs used in expr. It's similar to
// schemaexpr.ExtractColumnIDs but this function can also extract columns
// added in the same transaction (e.g. for `ADD COLUMN j INT CHECK (j > 0);`,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -410,7 +410,7 @@ func (izo *indexZoneConfigObj) fillIndexFromZoneSpecifier(b BuildCtx, zs tree.Zo
indexID = primaryIndexElem.IndexID
} else {
indexElems := b.ResolveIndex(tableID, tree.Name(indexName), ResolveParams{})
indexID = indexElems.FilterIndexName().MustGetOneElement().IndexID
indexID = indexElems.Filter(publicStatusFilter).FilterIndexName().MustGetOneElement().IndexID
}
izo.indexID = indexID
}
Loading

0 comments on commit 21d1123

Please sign in to comment.