Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
89014: jobs: Clear out claim info when pausing r=miretskiy a=miretskiy

Clear out job claim information when job is paused. Clearing out claim information is beneficial since it allows operator to pause/resume job if they want to try to move job coordinator to another node.

Addresses #82698

Release note: none

89026: kvserver: add `SmallEngineBlocks` testing knob and metamorphic params r=erikgrinaker a=erikgrinaker

`@cockroachdb/repl-prs` to do the main review, tagging other teams for visibility/review of metamorphic test params.

Resolves #86648.

---

**kvserver: add `SmallEngineBlocks` testing knob**

This patch adds a store testing knob `SmallEngineBlocks` which
configures Pebble with a block size of 1 byte. This will store every key
in a separate block, which can provoke bugs in time-bound iterators.

Release note: None
  
**sql/logictest: add metamorphic test param for small engine blocks**

Uses a Pebble block size of 1 byte, to provoke bugs in time-bound
iterators.

Release note: None
  
**kvserver/rangefeed: add metamorphic test param for small engine blocks**

Uses a Pebble block size of 1 byte, to provoke bugs in time-bound
iterators.

Release note: None
  
**kvserver/gc: add metamorphic test param for small engine blocks**

Uses a Pebble block size of 1 byte, to provoke bugs in time-bound
iterators.

Release note: None

  
**backupccl: add metamorphic test param for small engine blocks**

Uses a Pebble block size of 1 byte, to provoke bugs in time-bound
iterators.

Release note: None

89030: codeowners: add test-eng to owners of pkg/workload r=srosenberg a=srosenberg

Add test-eng as a code owner/watcher for pkg/workload.

In light of recent and future improvements [1], [2], TestEng would prefer to be in sync with all changes to the workload code. Over time, the team plans to build expertise in this area.

[1] #88362 [2] #88566

Release note: None
Release justification: test only change

Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Stan Rosenberg <[email protected]>
  • Loading branch information
4 people committed Sep 30, 2022
4 parents aaca5ce + b2a6b80 + 06b8405 + 1b0e197 commit 3e03176
Show file tree
Hide file tree
Showing 17 changed files with 158 additions and 21 deletions.
6 changes: 3 additions & 3 deletions .github/CODEOWNERS
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@
/pkg/ccl/testccl/workload/schemachange/ @cockroachdb/sql-schema
/pkg/ccl/testutilsccl/ @cockroachdb/test-eng-noreview
/pkg/ccl/utilccl/ @cockroachdb/unowned @cockroachdb/server-prs
/pkg/ccl/workloadccl/ @cockroachdb/sql-experience-noreview
/pkg/ccl/workloadccl/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng
/pkg/ccl/benchccl/rttanalysisccl/ @cockroachdb/sql-experience
/pkg/clusterversion/ @cockroachdb/kv-prs-noreview
/pkg/cmd/allocsim/ @cockroachdb/kv-prs
Expand Down Expand Up @@ -286,7 +286,7 @@
/pkg/cmd/uptodate/ @cockroachdb/dev-inf
/pkg/cmd/urlcheck/ @cockroachdb/docs
/pkg/cmd/whoownsit/ @cockroachdb/test-eng
/pkg/cmd/workload/ @cockroachdb/sql-experience-noreview
/pkg/cmd/workload/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng
/pkg/cmd/wraprules/ @cockroachdb/obs-inf-prs-noreview
/pkg/cmd/zerosum/ @cockroachdb/kv-noreview
/pkg/col/ @cockroachdb/sql-queries
Expand Down Expand Up @@ -360,7 +360,7 @@
/pkg/util/admission/ @cockroachdb/admission-control
/pkg/util/schedulerlatency/ @cockroachdb/admission-control
/pkg/util/tracing @cockroachdb/obs-inf-prs
/pkg/workload/ @cockroachdb/sql-experience-noreview
/pkg/workload/ @cockroachdb/sql-experience-noreview @cockroachdb/test-eng
/pkg/obs/ @cockroachdb/obs-inf-prs
/pkg/obsservice/ @cockroachdb/obs-inf-prs

Expand Down
24 changes: 24 additions & 0 deletions pkg/ccl/backupccl/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
Expand All @@ -60,6 +61,11 @@ const (
localFoo = "nodelocal://0/foo"
)

// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators. We disable this in race builds, which can
// be too slow.
var smallEngineBlocks = !util.RaceEnabled && util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)

func backupRestoreTestSetupWithParams(
t testing.TB,
clusterSize int,
Expand Down Expand Up @@ -90,6 +96,13 @@ func backupRestoreTestSetupWithParams(
}
}

if smallEngineBlocks {
if params.ServerArgs.Knobs.Store == nil {
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{}
}
params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true
}

tc = testcluster.StartTestCluster(t, clusterSize, params)
init(tc)

Expand Down Expand Up @@ -168,6 +181,14 @@ func backupRestoreTestSetupEmptyWithParams(
params.ServerArgsPerNode[i] = param
}
}

if smallEngineBlocks {
if params.ServerArgs.Knobs.Store == nil {
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{}
}
params.ServerArgs.Knobs.Store.(*kvserver.StoreTestingKnobs).SmallEngineBlocks = true
}

tc = testcluster.StartTestCluster(t, clusterSize, params)
init(tc)

Expand All @@ -192,6 +213,9 @@ func createEmptyCluster(
// Disabling the default test tenant due to test failures. More
// investigation is required. Tracked with #76378.
params.ServerArgs.DisableDefaultTestTenant = true
params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
}
tc := testcluster.StartTestCluster(t, clusterSize, params)

sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0])
Expand Down
4 changes: 3 additions & 1 deletion pkg/jobs/adopt.go
Original file line number Diff line number Diff line change
Expand Up @@ -432,7 +432,9 @@ const pauseAndCancelUpdate = `
ELSE status
END,
num_runs = 0,
last_run = NULL
last_run = NULL,
claim_session_id = NULL,
claim_instance_id = NULL
WHERE (status IN ('` + string(StatusPauseRequested) + `', '` + string(StatusCancelRequested) + `'))
AND ((claim_session_id = $1) AND (claim_instance_id = $2))
RETURNING id, status
Expand Down
25 changes: 25 additions & 0 deletions pkg/jobs/jobs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3138,9 +3138,31 @@ func TestPauseReason(t *testing.T) {
_, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */)
require.NoError(t, err)

countRowsWithClaimInfo := func() int {
t.Helper()
n := 0
tdb.QueryRow(t,
"SELECT count(*) FROM system.jobs "+
"WHERE id = $1 AND (claim_session_id IS NOT NULL OR claim_instance_id IS NOT NULL)",
jobID).Scan(&n)
return n
}
mustNotHaveClaim := func() {
require.Equal(t, 0, countRowsWithClaimInfo())
}
mustHaveClaim := func() {
testutils.SucceedsSoon(t, func() error {
if countRowsWithClaimInfo() == 1 {
return nil
}
return errors.New("still waiting for claim info")
})
}

// First wait until the job is running
q := fmt.Sprintf("SELECT status FROM system.jobs WHERE id = %d", jobID)
tdb.CheckQueryResultsRetry(t, q, [][]string{{"running"}})
mustHaveClaim()

getStatusAndPayload := func(t *testing.T, id jobspb.JobID) (string, jobspb.Payload) {
var payloadBytes []byte
Expand All @@ -3164,6 +3186,7 @@ func TestPauseReason(t *testing.T) {
require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "for testing"))
tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}})
checkStatusAndPauseReason(t, jobID, "paused", "for testing")
mustNotHaveClaim()
}

{
Expand All @@ -3172,12 +3195,14 @@ func TestPauseReason(t *testing.T) {
tdb.CheckQueryResultsRetry(t, q, [][]string{{"running"}})

checkStatusAndPauseReason(t, jobID, "running", "for testing")
mustHaveClaim()
}
{
// Pause the job again with a different reason. Verify that the job is paused with the reason.
require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "second time"))
tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}})
checkStatusAndPauseReason(t, jobID, "paused", "second time")
mustNotHaveClaim()
}
}

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/rangefeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/storageutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/ctxgroup",
"//pkg/util/encoding",
"//pkg/util/hlc",
Expand Down
25 changes: 23 additions & 2 deletions pkg/kv/kvclient/rangefeed/rangefeed_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/storageutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand All @@ -41,6 +42,10 @@ import (
var (
pointKV = storageutils.PointKV
rangeKV = storageutils.RangeKV

// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators.
smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)
)

type kvs = storageutils.KVs
Expand Down Expand Up @@ -564,7 +569,15 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
},
},
},
})
defer tc.Stopper().Stop(ctx)
srv := tc.Server(0)
db := srv.DB()
Expand Down Expand Up @@ -653,7 +666,15 @@ func TestWithOnDeleteRange(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
},
},
},
})
defer tc.Stopper().Stop(ctx)
srv := tc.Server(0)
db := srv.DB()
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/gc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ go_test(
"//pkg/base",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/rditer",
"//pkg/roachpb",
"//pkg/security/securityassets",
Expand All @@ -59,6 +60,7 @@ go_test(
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/leaktest",
Expand Down
10 changes: 10 additions & 0 deletions pkg/kv/kvserver/gc/gc_int_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"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"
Expand All @@ -29,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand All @@ -37,6 +39,11 @@ import (
"github.com/stretchr/testify/require"
)

// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators. We disable this under race, due to the slowdown.
var smallEngineBlocks = !util.RaceEnabled &&
util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)

func init() {
randutil.SeedForTests()
securityassets.SetLoader(securitytest.EmbeddedAssets)
Expand All @@ -54,6 +61,9 @@ func TestEndToEndGC(t *testing.T) {
tc := testcluster.NewTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
SmallEngineBlocks: smallEngineBlocks,
},
Server: &server.TestingKnobs{
WallClock: manualClock,
},
Expand Down
9 changes: 7 additions & 2 deletions pkg/kv/kvserver/gc/gc_random_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -87,6 +88,10 @@ var (
oldIntentFrac: .1,
rangeKeyFrac: .1,
}

// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators.
smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)
)

const intentAgeThreshold = 2 * time.Hour
Expand Down Expand Up @@ -121,7 +126,7 @@ func TestRunNewVsOld(t *testing.T) {
rng, seed := randutil.NewTestRand()
t.Logf("Using subtest seed: %d", seed)

eng := storage.NewDefaultInMemForTesting()
eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1)))
defer eng.Close()

tc.ds.dist(N, rng).setupTest(t, eng, *tc.ds.desc())
Expand Down Expand Up @@ -273,7 +278,7 @@ func TestNewVsInvariants(t *testing.T) {
t.Logf("Using subtest seed: %d", seed)

desc := tc.ds.desc()
eng := storage.NewDefaultInMemForTesting()
eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1)))
defer eng.Close()

sortedDistribution(tc.ds.dist(N, rng)).setupTest(t, eng, *desc)
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/rangefeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ go_test(
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/skip",
"//pkg/util",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/leaktest",
Expand Down
11 changes: 8 additions & 3 deletions pkg/kv/kvserver/rangefeed/catchup_scan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,18 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
)

// smallEngineBlocks configures Pebble with a block size of 1 byte, to provoke
// bugs in time-bound iterators.
var smallEngineBlocks = util.ConstantWithMetamorphicTestBool("small-engine-blocks", false)

// TODO(erikgrinaker): This should be migrated to a data-driven test harness for
// end-to-end rangefeed testing, with more exhaustive test cases. See:
// https://github.com/cockroachdb/cockroach/issues/82715
Expand Down Expand Up @@ -91,7 +96,7 @@ func TestCatchupScan(t *testing.T) {
kv2_2_2 := makeKTV(testKey2, ts2, testValue2)
kv2_5_3 := makeKTV(testKey2, ts5, testValue3)

eng := storage.NewDefaultInMemForTesting()
eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1)))
defer eng.Close()
// Put with no intent.
for _, kv := range []storage.MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1, kv2_2_2, kv2_5_3} {
Expand Down Expand Up @@ -141,7 +146,7 @@ func TestCatchupScanInlineError(t *testing.T) {
defer log.Scope(t).Close(t)

ctx := context.Background()
eng := storage.NewDefaultInMemForTesting()
eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1)))
defer eng.Close()

// Write an inline value.
Expand All @@ -167,7 +172,7 @@ func TestCatchupScanSeesOldIntent(t *testing.T) {
// [#85886]: https://github.com/cockroachdb/cockroach/issues/85886

ctx := context.Background()
eng := storage.NewDefaultInMemForTesting()
eng := storage.NewDefaultInMemForTesting(storage.If(smallEngineBlocks, storage.BlockSize(1)))
defer eng.Close()

// b -> version @ 1100 (visible)
Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -442,6 +442,11 @@ type StoreTestingKnobs struct {
// MVCCGCQueueLeaseCheckInterceptor intercepts calls to Replica.LeaseStatusAt when
// making high priority replica scans.
MVCCGCQueueLeaseCheckInterceptor func(ctx context.Context, replica *Replica, now hlc.ClockTimestamp) bool

// SmallEngineBlocks will configure the engine with a very small block size of
// 1 byte, resulting in each key having its own block. This can provoke bugs
// in time-bound iterators.
SmallEngineBlocks bool
}

// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.
Expand Down
Loading

0 comments on commit 3e03176

Please sign in to comment.