Skip to content

Commit

Permalink
Merge #106830
Browse files Browse the repository at this point in the history
106830: kvserver/gc: make TestEndToEndGC work with secondary tenants r=aliher1911 a=knz

Informs #76378 
Epic: CRDB-18499

Release note: None

Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
craig[bot] and knz committed Jul 17, 2023
2 parents c89d586 + 5e882f9 commit e0d4e62
Show file tree
Hide file tree
Showing 2 changed files with 67 additions and 32 deletions.
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/gc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ go_test(
embed = [":gc"],
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvpb",
Expand All @@ -57,6 +58,8 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/server/serverpb",
"//pkg/settings/cluster",
"//pkg/sql/sqlliveness/slinstance",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/testutils",
Expand Down
96 changes: 64 additions & 32 deletions pkg/kv/kvserver/gc/gc_int_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,16 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slinstance"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -53,7 +56,7 @@ func init() {

func TestEndToEndGC(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
defer ccl.TestingEnableEnterprise()()

for _, d := range []struct {
// Using range tombstones to remove data will promote full range deletions
Expand All @@ -79,36 +82,59 @@ func TestEndToEndGC(t *testing.T) {
} {
t.Run(fmt.Sprintf("rangeTombstones=%t/clearRange=%t", d.rangeTombstones, d.clearRange),
func(t *testing.T) {
defer log.Scope(t).Close(t)
ctx := context.Background()

settings := cluster.MakeTestingClusterSettings()
// Push the TTL up to 60 hours since we emulate a 50 hours
// clock jump below.
slinstance.DefaultTTL.Override(ctx, &settings.SV, 60*time.Hour)

manualClock := hlc.NewHybridManualClock()
tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
},
Server: &server.TestingKnobs{
WallClock: manualClock,
},
s, appSqlDb, appKvDb := serverutils.StartServer(t, base.TestServerArgs{
Settings: settings,
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
},
Server: &server.TestingKnobs{
WallClock: manualClock,
},
DefaultTestTenant: base.TODOTestTenantDisabled,
},
})
tc.Start(t)
defer tc.Stopper().Stop(context.Background())
require.NoError(t, tc.WaitForFullReplication())
defer s.Stopper().Stop(ctx)

sqlDb := tc.ServerConn(0)
kvDb := tc.Server(0).DB()
statusServer := tc.Server(0).StatusServer().(serverpb.StatusServer)
statusServer := s.StatusServer().(serverpb.StatusServer)
systemSqlDb := serverutils.OpenDBConn(t, s.SQLAddr(), "system", false, s.Stopper())

execOrFatal := func(t *testing.T, db *gosql.DB, stmt string, args ...interface{}) {
t.Helper()
_, err := db.Exec(stmt, args...)
require.NoError(t, err, "failed to execute %s", stmt)
}

waitForTableSplit := func(t *testing.T, db *gosql.DB) {
t.Helper()
testutils.SucceedsSoon(t, func() error {
// List the number of tables that share a range with our test
// 'kv' table. If the table has been split, there should be
// just 1 table (the 'kv' table itself).
row := db.QueryRow(`
WITH ranges_tables AS (SHOW CLUSTER RANGES WITH TABLES)
SELECT count(*) FROM ranges_tables t1
WHERE 'kv' IN (
SELECT table_name FROM ranges_tables t2 WHERE t1.range_id = t2.range_id
)`)
require.NoError(t, row.Err(), "failed to query ranges")
var numTables int
require.NoError(t, row.Scan(&numTables), "failed to read row with range id")
if numTables > 1 {
return errors.Newf("%d table(s) not split yet", numTables)
}
return nil
})
}

getTableRangeIDs := func(t *testing.T, db *gosql.DB) ids {
t.Helper()
rows, err := db.Query("WITH r AS (SHOW RANGES FROM TABLE kv) SELECT range_id FROM r ORDER BY start_key")
Expand Down Expand Up @@ -148,7 +174,7 @@ func TestEndToEndGC(t *testing.T) {
t.Helper()
var prevRangeIDs ids
for i := 0; i < 3; i++ {
rangeIDs := getTableRangeIDs(t, sqlDb)
rangeIDs := getTableRangeIDs(t, appSqlDb)
if rangeIDs.equal(prevRangeIDs) {
return
}
Expand Down Expand Up @@ -194,22 +220,28 @@ func TestEndToEndGC(t *testing.T) {
// Set closed timestamp duration, this is needed to avoid waiting for default
// 2 min interval for protected timestamp to get bumped and letting GC collect
// old values.
execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.protectedts.poll_interval = '5s'`)
execOrFatal(t, systemSqlDb, `SET CLUSTER SETTING kv.protectedts.poll_interval = '5s'`)
execOrFatal(t, appSqlDb, `SET CLUSTER SETTING kv.protectedts.poll_interval = '5s'`)

// Ensure that each table gets its own range.
execOrFatal(t, systemSqlDb, `SET CLUSTER SETTING spanconfig.tenant_coalesce_adjacent.enabled = 'false'`)
execOrFatal(t, systemSqlDb, `SET CLUSTER SETTING spanconfig.storage_coalesce_adjacent.enabled = 'false'`)

if d.clearRange {
execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 5`)
execOrFatal(t, systemSqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 5`)
} else {
execOrFatal(t, sqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 0`)
execOrFatal(t, systemSqlDb, `SET CLUSTER SETTING kv.gc.clear_range_min_keys = 0`)
}

execOrFatal(t, sqlDb, `CREATE TABLE kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`)
execOrFatal(t, appSqlDb, `CREATE TABLE kv (k BIGINT NOT NULL PRIMARY KEY, v BYTES NOT NULL)`)

for i := 0; i < 1000; i++ {
execOrFatal(t, sqlDb, "UPSERT INTO kv VALUES ($1, $2)", rng.Int63(), "hello")
execOrFatal(t, appSqlDb, "UPSERT INTO kv VALUES ($1, $2)", rng.Int63(), "hello")
}
t.Logf("found table range after initializing table data: %s", getTableRangeIDs(t, sqlDb))
waitForTableSplit(t, appSqlDb)
t.Logf("found table range after initializing table data: %s", getTableRangeIDs(t, appSqlDb))

require.NotEmptyf(t, readSomeKeys(t, sqlDb), "found no keys in table")
require.NotEmptyf(t, readSomeKeys(t, appSqlDb), "found no keys in table")

// Since ranges query and checking range stats are non atomic there could be
// a range split/merge operation caught in between. That could produce empty
Expand All @@ -219,7 +251,7 @@ func TestEndToEndGC(t *testing.T) {
const tableRangesRetry = 3
var tableRangeIDs, nonEmptyRangeIDs ids
for i := 0; i < tableRangesRetry; i++ {
tableRangeIDs = getTableRangeIDs(t, sqlDb)
tableRangeIDs = getTableRangeIDs(t, appSqlDb)
if len(tableRangeIDs) == 0 {
continue
}
Expand All @@ -234,13 +266,13 @@ func TestEndToEndGC(t *testing.T) {
t.Logf("found non-empty table ranges before deletion: %v", nonEmptyRangeIDs)

if d.rangeTombstones {
deleteRangeDataWithRangeTombstone(t, kvDb, sqlDb)
deleteRangeDataWithRangeTombstone(t, appKvDb, appSqlDb)
} else {
execOrFatal(t, sqlDb, "DELETE FROM kv WHERE k IS NOT NULL")
execOrFatal(t, appSqlDb, "DELETE FROM kv WHERE k IS NOT NULL")
}
t.Logf("found table ranges after range deletion: %s", getTableRangeIDs(t, sqlDb))
t.Logf("found table ranges after range deletion: %s", getTableRangeIDs(t, appSqlDb))

require.Empty(t, readSomeKeys(t, sqlDb), "table still contains data after range deletion")
require.Empty(t, readSomeKeys(t, appSqlDb), "table still contains data after range deletion")

// Push clock forward to make all data eligible for GC. Mind that this is not
// enough just to push the clock, we need to wait for protected timestamp to
Expand All @@ -253,11 +285,11 @@ func TestEndToEndGC(t *testing.T) {
// processing as we could only enqueue, but not force GC op.
enqueueSucceeded := false
testutils.SucceedsSoon(t, func() error {
tableRangeIDs := getTableRangeIDs(t, sqlDb)
tableRangeIDs := getTableRangeIDs(t, appSqlDb)
t.Logf("pushing kv table ranges through mvcc gc queue: %s", tableRangeIDs)

for _, id := range tableRangeIDs {
_, err := sqlDb.Exec(`SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, id)
_, err := systemSqlDb.Exec(`SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, id)
if err != nil {
t.Logf("failed to enqueue range to mvcc gc queue: %s", err)
}
Expand Down

0 comments on commit e0d4e62

Please sign in to comment.