Skip to content

Commit

Permalink
Merge #105533
Browse files Browse the repository at this point in the history
105533: sql: enable TestDropTableDeleteData test r=chengxiong-ruan a=chengxiong-ruan

This commit re-enables TestDropTableDeleteData test by utilizing the mvcc gc queue settings.

Informs: #85876

Release note: None

Co-authored-by: Chengxiong Ruan <[email protected]>
  • Loading branch information
craig[bot] and chengxiong-ruan committed Jun 28, 2023
2 parents 2689002 + a70d78f commit 4477505
Show file tree
Hide file tree
Showing 3 changed files with 66 additions and 49 deletions.
74 changes: 25 additions & 49 deletions pkg/sql/drop_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,10 @@ import (
"context"
gosql "database/sql"
"fmt"
"math/rand"
"net/url"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
Expand Down Expand Up @@ -679,19 +679,19 @@ func TestDropTable(t *testing.T) {
func TestDropTableDeleteData(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
skip.WithIssue(t, 85876)
params, _ := tests.CreateTestServerParams()

defer gcjob.SetSmallMaxGCIntervalForTest()()
// Speed up mvcc queue scan.
params.ScanMaxIdleTime = time.Millisecond

s, sqlDB, kvDB := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())
ctx := context.Background()

_, err := sqlDB.Exec(`SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '1s';`)
require.NoError(t, err)

// Disable strict GC TTL enforcement because we're going to shove a zero-value
// TTL into the system with AddImmediateGCZoneConfig.
defer sqltestutils.DisableGCTTLStrictEnforcement(t, sqlDB)()
sqltestutils.SetShortRangeFeedIntervals(t, sqlDB)

const numRows = 2*row.TableTruncateChunkSize + 1
const numKeys = 3 * numRows
Expand Down Expand Up @@ -723,7 +723,14 @@ func TestDropTableDeleteData(t *testing.T) {

tableSpan := descs[i].TableSpan(keys.SystemSQLCodec)
tests.CheckKeyCount(t, kvDB, tableSpan, numKeys)
if _, err := sqlDB.Exec(fmt.Sprintf(`DROP TABLE t.%s`, tableName)); err != nil {

if _, err := sqltestutils.AddImmediateGCZoneConfig(sqlDB, descs[i].GetID()); err != nil {
t.Fatal(err)
}
}

for i := 0; i < numTables; i++ {
if _, err := sqlDB.Exec(fmt.Sprintf(`DROP TABLE t.%s`, descs[i].GetName())); err != nil {
t.Fatal(err)
}
}
Expand All @@ -735,54 +742,45 @@ func TestDropTableDeleteData(t *testing.T) {
t.Fatal(err)
}
tableSpan := descs[i].TableSpan(keys.SystemSQLCodec)
tests.CheckKeyCount(t, kvDB, tableSpan, numKeys)
tests.CheckKeyCountIncludingTombstoned(t, s, tableSpan, numKeys)

if err := jobutils.VerifySystemJob(t, sqlRun, i,
if err := jobutils.VerifySystemJob(t, sqlRun, numTables+i,
jobspb.TypeNewSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: username.RootUserName(),
Description: fmt.Sprintf(`DROP TABLE t.public.%s`, descs[i].GetName()),
DescriptorIDs: descpb.IDs{
descs[i].GetID(),
},
}); err != nil {
t.Fatal(err)
}
}

// The closure pushes a zone config reducing the TTL to 0 for descriptor i.
pushZoneCfg := func(i int) {
if _, err := sqltestutils.AddImmediateGCZoneConfig(sqlDB, descs[i].GetID()); err != nil {
t.Fatal(err)
}
}
// Refresh protected timestamp cache immediately to make MVCC GC queue to
// process GC immediately.
_, err = sqlDB.Exec(`SET CLUSTER SETTING kv.protectedts.poll_interval = '1s';`)
require.NoError(t, err)

checkTableGCed := func(i int) {
testutils.SucceedsSoon(t, func() error {
if err := descExists(sqlDB, false, descs[i].GetID()); err != nil {
return err
}

return zoneExists(sqlDB, nil, descs[i].GetID())
})

tableSpan := descs[i].TableSpan(keys.SystemSQLCodec)
tests.CheckKeyCount(t, kvDB, tableSpan, 0)
tests.CheckKeyCountIncludingTombstoned(t, s, tableSpan, 0)

// Ensure that the job is marked as succeeded.
if err := jobutils.VerifySystemJob(t, sqlRun, i,
if err := jobutils.VerifySystemJob(t, sqlRun, numTables+i,
jobspb.TypeNewSchemaChange, jobs.StatusSucceeded, jobs.Record{
Username: username.RootUserName(),
Description: fmt.Sprintf(`DROP TABLE t.public.%s`, descs[i].GetName()),
DescriptorIDs: descpb.IDs{
descs[i].GetID(),
},
}); err != nil {
t.Fatal(err)
}

// Ensure that the job is marked as succeeded.
// Note that this 2*i+1 nonsense comes
// Ensure that the gc job is marked as succeeded.
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, 2*i+1,
return jobutils.VerifySystemJob(t, sqlRun, numTables+i,
jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{
Username: username.NodeUserName(),
Description: fmt.Sprintf(`GC for DROP TABLE t.public.%s`, descs[i].GetName()),
Expand All @@ -793,29 +791,7 @@ func TestDropTableDeleteData(t *testing.T) {
})
}

// Push a new zone config for a few tables with TTL=0 so the data
// is deleted immediately.
barrier := rand.Intn(numTables)
for i := 0; i < barrier; i++ {
pushZoneCfg(i)
}

// Check GC worked!
for i := 0; i < numTables; i++ {
if i < barrier {
checkTableGCed(i)
} else {
// Data still present for tables past barrier.
tableSpan := descs[i].TableSpan(keys.SystemSQLCodec)
tests.CheckKeyCount(t, kvDB, tableSpan, numKeys)
}
}

// Push the rest of the zone configs and check all the data gets GC-ed.
for i := barrier; i < numTables; i++ {
pushZoneCfg(i)
}
for i := barrier; i < numTables; i++ {
checkTableGCed(i)
}
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ go_library(
"//pkg/roachpb",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlstats",
"//pkg/storage",
"//pkg/testutils/serverutils",
"//pkg/testutils/storageutils",
"//pkg/util/syncutil",
"@com_github_cockroachdb_errors//:errors",
Expand Down
39 changes: 39 additions & 0 deletions pkg/sql/tests/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ import (

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/errors"
)

Expand All @@ -31,6 +33,43 @@ func CheckKeyCount(t *testing.T, kvDB *kv.DB, span roachpb.Span, numKeys int) {
}
}

// CheckKeyCountIncludingTombstoned checks that the number of keys (including
// those whose tombstones are marked but not GC'ed yet) in the provided span
// matches the expected number.
func CheckKeyCountIncludingTombstoned(
t *testing.T, s serverutils.TestServerInterface, tableSpan roachpb.Span, expectedNum int,
) {
// Check key count including tombstoned ones.
engines := s.Engines()
if len(engines) != 1 {
t.Fatalf("expecting 1 engine from the test server, but found %d", len(engines))
}

keyCount := 0
it := engines[0].NewMVCCIterator(
storage.MVCCKeyIterKind,
storage.IterOptions{
LowerBound: tableSpan.Key,
UpperBound: tableSpan.EndKey,
},
)

for it.SeekGE(storage.MVCCKey{Key: tableSpan.Key}); ; it.NextKey() {
ok, err := it.Valid()
if err != nil {
t.Fatal(err)
}
if !ok {
break
}
keyCount++
}
it.Close()
if keyCount != expectedNum {
t.Fatalf("expecting %d keys, but found %d", expectedNum, keyCount)
}
}

// CheckKeyCountE returns an error if the the number of keys in the
// provided span does not match numKeys.
func CheckKeyCountE(t *testing.T, kvDB *kv.DB, span roachpb.Span, numKeys int) error {
Expand Down

0 comments on commit 4477505

Please sign in to comment.