Skip to content

Commit

Permalink
Merge #78472
Browse files Browse the repository at this point in the history
78472: sql: allow zone configuration cleanup in tenants r=arulajmani a=stevendanna

Previously RemoveIndexZoneConfigs early-exited if it was run from a
tenant since zone configs weren't supported in tenants. Now, we do
support zone configurations in tenants and the early exit would result
in zone configurations not being removed when an index was dropped.

Release justification: Low risk bug fix for new functionality

Release note: None

Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
craig[bot] and stevendanna committed Mar 30, 2022
2 parents bd80d60 + 5445e25 commit 350b22f
Show file tree
Hide file tree
Showing 5 changed files with 121 additions and 6 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_test(
"//pkg/ccl/partitionccl",
"//pkg/ccl/utilccl",
"//pkg/config/zonepb",
"//pkg/jobs/jobspb",
"//pkg/kv",
"//pkg/roachpb",
"//pkg/security",
Expand All @@ -33,6 +34,7 @@ go_test(
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/syncutil",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_stretchr_testify//require",
],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,14 @@ import (
"fmt"
"sort"
"strings"
"sync"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
Expand All @@ -34,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -80,6 +83,18 @@ func TestDataDriven(t *testing.T) {

ctx := context.Background()

gcWaiter := sync.NewCond(&syncutil.Mutex{})
allowGC := true
gcTestingKnobs := &sql.GCJobTestingKnobs{
RunBeforeResume: func(_ jobspb.JobID) error {
gcWaiter.L.Lock()
for !allowGC {
gcWaiter.Wait()
}
gcWaiter.L.Unlock()
return nil
},
}
scKnobs := &spanconfig.TestingKnobs{
// Instead of relying on the GC job to wait out TTLs and clear out
// descriptors, let's simply exclude dropped tables to simulate
Expand All @@ -94,6 +109,7 @@ func TestDataDriven(t *testing.T) {
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
GCJob: gcTestingKnobs,
SpanConfig: scKnobs,
},
},
Expand Down Expand Up @@ -239,6 +255,17 @@ func TestDataDriven(t *testing.T) {
var recordID string
d.ScanArgs(t, "record-id", &recordID)
tenant.ReleaseProtectedTimestampRecord(ctx, recordID)

case "block-gc-jobs":
gcWaiter.L.Lock()
allowGC = false
gcWaiter.L.Unlock()

case "unblock-gc-jobs":
gcWaiter.L.Lock()
allowGC = true
gcWaiter.Signal()
gcWaiter.L.Unlock()
default:
t.Fatalf("unknown command: %s", d.Cmd)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,3 +92,85 @@ translate database=db table=t
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/10{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

block-gc-jobs
----

# Create an index and create a zone configuration in the same transaction
# with the mvcc-index-backfiller.
exec-sql
SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = true;
CREATE INDEX idx2 ON db.t (j);
ALTER INDEX db.t@idx2 CONFIGURE ZONE USING gc.ttlseconds = 1;
----

# Both the newly added index and the temporary index have the configured zone configuration.
translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/106/{5-6} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/6-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

unblock-gc-jobs
----

exec-sql
SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS])
----

# The zone configuration for the temporary index is cleaned up
translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/5-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7

# Create an index and create a zone configuration in the same transaction
# with the non-mvcc-index-backfiller.
exec-sql
SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false;
CREATE INDEX idx3 ON db.t (j);
ALTER INDEX db.t@idx3 CONFIGURE ZONE USING gc.ttlseconds = 1;
----

translate database=db table=t
----
/Tenant/10/Table/106{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5
/Tenant/10/Table/106/{3-4} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{4-5} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/106/{5-6} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7
/Tenant/10/Table/106/{6-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=1 num_replicas=7
/Tenant/10/Table/10{6/7-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7


# Create and drop an index inside the same transaction. The related
# zone configuration should also be cleaned up.
exec-sql
SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = true;
CREATE TABLE db.t2(i INT PRIMARY KEY, j INT);
CREATE INDEX idx ON db.t2 (j);
ALTER INDEX db.t2@idx CONFIGURE ZONE USING gc.ttlseconds = 1;
DROP INDEX db.t2@idx
----

translate database=db table=t2
----
/Tenant/10/Table/10{7-8} ttl_seconds=3600 num_replicas=7

exec-sql
SET CLUSTER SETTING sql.mvcc_compliant_index_creation.enabled = false;
CREATE TABLE db.t3(i INT PRIMARY KEY, j INT);
CREATE INDEX idx ON db.t3 (j);
ALTER INDEX db.t3@idx CONFIGURE ZONE USING gc.ttlseconds = 1;
DROP INDEX db.t3@idx
----

translate database=db table=t3
----
/Tenant/10/Table/10{8-9} ttl_seconds=3600 num_replicas=7
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,15 @@ CREATE INDEX idx ON db.t1(id);
ALTER INDEX db.t1@idx CONFIGURE ZONE USING gc.ttlseconds = 1;
----

# We wait for running jobs to complete to ensure that index GC jobs
# have completed. Index GC jobs are enqueued for temporary indexes
# used by the mvcc-compatible index backfiller. If we didn't wait for
# them to complete, our span configuration may still contain entries
# for the temporary index.
exec-sql
SHOW JOBS WHEN COMPLETE (SELECT job_id FROM [SHOW JOBS])
----

protect record-id=5 ts=5
descs 106
----
Expand All @@ -107,6 +116,5 @@ translate database=db
----
/Tenant/10/Table/106{-/2} num_replicas=7 num_voters=5 protection_policies=[{ts: 5}]
/Tenant/10/Table/106/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 protection_policies=[{ts: 5}]
/Tenant/10/Table/106/{3-4} ttl_seconds=1 num_replicas=7 num_voters=5 protection_policies=[{ts: 5}]
/Tenant/10/Table/10{6/4-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 5}]
/Tenant/10/Table/10{6/3-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 5}]
/Tenant/10/Table/10{7-8} num_replicas=7
4 changes: 0 additions & 4 deletions pkg/sql/set_zone_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -1182,10 +1182,6 @@ func RemoveIndexZoneConfigs(
tableDesc catalog.TableDescriptor,
indexIDs []uint32,
) error {
if !execCfg.Codec.ForSystemTenant() {
// Tenants are agnostic to zone configs.
return nil
}
zone, err := getZoneConfigRaw(ctx, txn, execCfg.Codec, execCfg.Settings, tableDesc.GetID())
if err != nil {
return err
Expand Down

0 comments on commit 350b22f

Please sign in to comment.