Skip to content

Commit ee166ea

Browse files
craig[bot]tbgstevendanna
committed
153401: batcheval: reliably remove GCHint when it expires r=tbg a=tbg The GCHint would previously only be adjusted when GC would actually do work. However, conceivably (and reproducably), sometimes all of the GC work would be done under a GCThreshold that did not yet allow the GCHint to be collected. This would leave the GCHint dangling, and endlessly queue the range for GC over and over again. Unconditionally clear the GCHint to avoid this. The problem could reliably be reproduced using an [experiment]. It no longer reproduces as of this commit. [experiment]: https://docs.google.com/document/d/1QajDmDPgICvwYeRmrFVKvqVRW_kwWMD5RAe_q6irCXc/edit?tab=t.0 See also this [Slack thread]. Found while investigating https://github.com/cockroachlabs/support/issues/3424. #153564 was filed as a result of looking further into a log message involved in this investigation leading up to this PR. [Slack thread]: https://cockroachlabs.slack.com/archives/G01G8LK77DK/p1757580849424569?thread_ts=1757428752.120299&cid=G01G8LK77DK Epic: none 153526: kvcoord: reset HasBufferedAllPrecedingWrites for multi-batch flush r=miraradeva a=stevendanna Fixes #153509 Release note: None 153551: roachtest: make gossip chaos test more resilient r=tbg a=tbg See #153403 (comment). Via backport: Closes #153403. Epic: none 153552: roachtest: make failure in follower-reads more helpful r=tbg a=tbg ... by telling the reader what to look for to dig in. Improves #153404. Epic: none 153553: roachtest: bump vmodule in gossip/restart-node-one r=tbg a=tbg See #153441. Epic: none Co-authored-by: Tobias Grieger <[email protected]> Co-authored-by: Steven Danna <[email protected]>
6 parents 6792705 + 765504b + 142f984 + 04db75f + 18b6054 + c40a33b commit ee166ea

File tree

6 files changed

+72
-7
lines changed

6 files changed

+72
-7
lines changed

pkg/cmd/roachtest/tests/follower_reads.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -792,7 +792,8 @@ func verifySQLLatency(
792792
}
793793
}
794794
if permitted := int(.2 * float64(len(perTenSeconds))); len(above) > permitted {
795-
t.Fatalf("%d latency values (%v) are above target latency %v, %d permitted",
795+
t.Fatalf("%d latency values (%v) are above target latency %v, %d permitted "+
796+
`(search the cluster logs for 'SELECT v FROM "".mr_db.test WHERE k = $1' to look at the traces)`,
796797
len(above), above, targetLatency, permitted)
797798
}
798799
}

pkg/cmd/roachtest/tests/gossip.go

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import (
2929
"github.com/cockroachdb/cockroach/pkg/roachprod"
3030
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
3131
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
32+
"github.com/cockroachdb/cockroach/pkg/testutils"
3233
"github.com/cockroachdb/cockroach/pkg/util"
3334
"github.com/cockroachdb/cockroach/pkg/util/allstacks"
3435
"github.com/cockroachdb/cockroach/pkg/util/retry"
@@ -60,7 +61,17 @@ SELECT node_id
6061
tBeforePing := timeutil.Now()
6162
db := c.Conn(ctx, t.L(), node)
6263
defer db.Close()
63-
require.NoError(t, db.Ping())
64+
65+
testutils.SucceedsSoon(t, func() error {
66+
// Having just shut down a node, the sql user table may be in the
67+
// process of failing over, and if we're unlucky and try to open a new
68+
// conn here, we can sometimes hit an internal 10s timeout should the
69+
// failover take longer than usual.
70+
//
71+
// See https://github.com/cockroachdb/cockroach/issues/153403#issuecomment-3296381756.
72+
return db.Ping()
73+
})
74+
6475
tAfterPing := timeutil.Now()
6576
if pingDur := tAfterPing.Sub(tBeforePing); pingDur > 20*time.Second {
6677
t.L().Printf("sql connection ready after %.2fs", pingDur.Seconds())
@@ -387,8 +398,10 @@ func runGossipRestartNodeOne(ctx context.Context, t test.Test, c cluster.Cluster
387398
// Reduce the scan max idle time to speed up evacuation of node 1.
388399
settings := install.MakeClusterSettings(install.NumRacksOption(c.Spec().NodeCount))
389400
settings.Env = append(settings.Env, "COCKROACH_SCAN_MAX_IDLE_TIME=5ms")
401+
startOpts := option.DefaultStartOpts()
402+
startOpts.RoachprodOpts.ExtraArgs = []string{"--vmodule=*=1"} // see #153441
390403

391-
c.Start(ctx, t.L(), option.DefaultStartOpts(), settings)
404+
c.Start(ctx, t.L(), startOpts, settings)
392405

393406
db := c.Conn(ctx, t.L(), 1)
394407
defer db.Close()
@@ -542,7 +555,8 @@ SELECT count(replicas)
542555
c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.Node(1))
543556
// N.B. Since n1 was initially stripped of all the replicas, we must wait for full replication. Otherwise, the
544557
// replica consistency checks may time out.
545-
c.Start(ctx, t.L(), option.NewStartOpts(option.WaitForReplication()), install.MakeClusterSettings(), c.Node(1))
558+
startOpts.WaitForReplicationFactor = 3
559+
c.Start(ctx, t.L(), startOpts, install.MakeClusterSettings(), c.Node(1))
546560
}
547561

548562
func runCheckLocalityIPAddress(ctx context.Context, t test.Test, c cluster.Cluster) {

pkg/kv/kvclient/kvcoord/txn_interceptor_write_buffer.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1826,6 +1826,9 @@ func (twb *txnWriteBuffer) flushBufferAndSendBatch(
18261826
if err := requireAllFlushedRequestsProcessed(br.Responses); err != nil {
18271827
return nil, kvpb.NewError(err)
18281828
}
1829+
1830+
// We've written intents now, so this should be false.
1831+
ba.HasBufferedAllPrecedingWrites = false
18291832
ba.UpdateTxn(br.Txn)
18301833

18311834
return twb.wrapped.SendLocked(ctx, ba)

pkg/kv/kvclient/kvcoord/txn_interceptor_write_buffer_test.go

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2671,6 +2671,49 @@ func TestTxnWriteBufferHasBufferedAllPrecedingWrites(t *testing.T) {
26712671
}
26722672
}
26732673

2674+
func TestTxnWriteBufferHasBufferedAllPrecedingWritesSplitFlush(t *testing.T) {
2675+
ctx := context.Background()
2676+
twb, mockSender, _ := makeMockTxnWriteBuffer(ctx)
2677+
txn := makeTxnProto()
2678+
txn.Sequence = 1
2679+
keyA, keyB, keyC := roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c")
2680+
2681+
// This Put will be completely buffered.
2682+
ba := &kvpb.BatchRequest{Header: kvpb.Header{Txn: &txn}}
2683+
putA := putArgs(keyA, "valA", txn.Sequence)
2684+
ba.Add(putA)
2685+
2686+
br, pErr := twb.SendLocked(ctx, ba)
2687+
require.Nil(t, pErr)
2688+
require.NotNil(t, br)
2689+
2690+
// Send a DeleteRange with MaxSpanRequestKeys set which will force a flush
2691+
// using two batches. The first should have HasBufferedAllPrecedingWrites, the
2692+
// next should not.
2693+
reqCount := 0
2694+
mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) {
2695+
require.Equal(t, reqCount == 0, ba.HasBufferedAllPrecedingWrites,
2696+
"HasBufferedAllPrecedingWrites expected on the first request (and only the first request")
2697+
reqCount++
2698+
br = ba.CreateReply()
2699+
br.Txn = ba.Txn
2700+
return br, nil
2701+
})
2702+
2703+
txn.Sequence++
2704+
ba = &kvpb.BatchRequest{Header: kvpb.Header{Txn: &txn}}
2705+
ba.MaxSpanRequestKeys = 10
2706+
ba.Add(delRangeArgs(keyB, keyC, txn.Sequence))
2707+
2708+
beforeCallCount := mockSender.NumCalled()
2709+
br, pErr = twb.SendLocked(ctx, ba)
2710+
require.Nil(t, pErr)
2711+
require.NotNil(t, br)
2712+
require.Equal(t, 2, mockSender.NumCalled()-beforeCallCount)
2713+
require.Len(t, br.Responses, 1)
2714+
require.IsType(t, &kvpb.DeleteRangeResponse{}, br.Responses[0].GetInner())
2715+
}
2716+
26742717
// BenchmarkTxnWriteBuffer benchmarks the txnWriteBuffer. The test sets up a
26752718
// transaction with an existing buffer and runs a single batch through
26762719
// SendLocked and flushBufferAndSendBatch. The test varies the state of the

pkg/kv/kvserver/batcheval/cmd_gc.go

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -265,9 +265,7 @@ func GC(
265265
// Check if optional GC hint on the range is expired (e.g. delete operation is
266266
// older than GC threshold) and remove it. Otherwise this range could be
267267
// unnecessarily GC'd with high priority again.
268-
// We should only do that when we are doing actual cleanup as we want to have
269-
// a hint when request is being handled.
270-
if len(args.Keys) != 0 || len(args.RangeKeys) != 0 || args.ClearRange != nil {
268+
{
271269
sl := MakeStateLoader(cArgs.EvalCtx)
272270
hint, err := sl.LoadGCHint(ctx, readWriter)
273271
if err != nil {

pkg/kv/kvserver/mvcc_gc_queue.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -783,6 +783,12 @@ func (mgcq *mvccGCQueue) process(
783783
if scoreAfter.ShouldQueue {
784784
// The scores are very long, so splitting into multiple lines manually for
785785
// readability.
786+
//
787+
// NB: there are likely situations in which this check triggers incorrectly,
788+
// for example when the GC hint triggers GC but a protected timestamp
789+
// prevents the GC threshold from advancing. In that case, not only did we
790+
// run a GC cycle without improving anything, but we also pile up a stats
791+
// recomputation. This is hopefully too rare to matter.
786792
log.Dev.Infof(ctx, "GC still needed following GC, recomputing MVCC stats")
787793
log.Dev.Infof(ctx, "old score %s", r)
788794
log.Dev.Infof(ctx, "new score %s", scoreAfter)

0 commit comments

Comments
 (0)