From 93b815fba8b629b7efebfa63f807cb843a94bb5c Mon Sep 17 00:00:00 2001 From: arulajmani Date: Fri, 21 Jan 2022 12:57:30 -0500 Subject: [PATCH 1/6] kvserver: make AdminVerifyProtectedTimestamp a no-op As part of re-working the PTS subsystem to work with secondary tenants we are dropping verification of protection records. None of the existing users of `Verification` (currently only backup) would fail non-destructively without it; this allows us to circumvent the complexity involved in making `Verification` work in the new subsystem. This patch removes code that was previously used to serve the `AdminVerifyProtectedTimestamp` request; we instead vacuously return true. This is to account for the {21.2, 22.1} mixed version case where 22.1 binary nodes still need to be able to serve this request. This can happen if the request is initiated on a 21.2 node and the leaseholder of the range it is trying to verify is on a 22.1 node. By always returning true we ensure the (backup) job upstream doesn't fail. This is okay even if the PTS record being verified does not apply as the failure mode is non-destructive (which is also why we're okay removing the call to verification in 22.1). This patch is required in preperation for the `ProtectedTSReader`. Release note: None Release justification: low risk, high benefit changes to existing functionality. --- pkg/kv/kvserver/BUILD.bazel | 1 - pkg/kv/kvserver/client_protectedts_test.go | 55 ++- pkg/kv/kvserver/client_replica_test.go | 19 +- pkg/kv/kvserver/replica_command.go | 42 +- .../kvserver/replica_protected_timestamp.go | 154 -------- .../replica_protected_timestamp_test.go | 371 ------------------ 6 files changed, 75 insertions(+), 567 deletions(-) diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index e24b2673dcf9..ecf5ab794434 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -339,7 +339,6 @@ go_test( "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/kv/kvserver/protectedts/ptstorage", - "//pkg/kv/kvserver/protectedts/ptverifier", "//pkg/kv/kvserver/raftentry", "//pkg/kv/kvserver/rditer", "//pkg/kv/kvserver/readsummary/rspb", diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index da28e02c3ef9..ce55646c9696 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptverifier" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -210,12 +210,13 @@ func TestProtectedTimestamps(t *testing.T) { thresh := thresholdFromTrace(trace) require.Truef(t, thresh.Less(ptsRec.Timestamp), "threshold: %v, protected %v %q", thresh, ptsRec.Timestamp, trace) - // Verify that the record indeed did apply as far as the replica is concerned. - ptv := ptverifier.New(s0.DB(), pts) - require.NoError(t, ptv.Verify(ctx, ptsRec.ID.GetUUID())) - ptsRecVerified, err := ptsWithDB.GetRecord(ctx, nil /* txn */, ptsRec.ID.GetUUID()) - require.NoError(t, err) - require.True(t, ptsRecVerified.Verified) + // Verify that the record did indeed make its way down into KV where the + // replica can read it from. + ptp := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider + require.NoError( + t, + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, ptsRec.Timestamp, ptsRec.DeprecatedSpans), + ) // Make a new record that is doomed to fail. failedRec := ptsRec @@ -226,9 +227,12 @@ func TestProtectedTimestamps(t *testing.T) { _, err = ptsWithDB.GetRecord(ctx, nil /* txn */, failedRec.ID.GetUUID()) require.NoError(t, err) - // Verify that it indeed did fail. - verifyErr := ptv.Verify(ctx, failedRec.ID.GetUUID()) - require.Regexp(t, "failed to verify protection", verifyErr) + // replica can read it from. We then verify (below) that the failed record + // does not affect the ability to GC. + require.NoError( + t, + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, failedRec.Timestamp, failedRec.DeprecatedSpans), + ) // Add a new record that is after the old record. laterRec := ptsRec @@ -236,7 +240,10 @@ func TestProtectedTimestamps(t *testing.T) { laterRec.Timestamp = afterWrites laterRec.Timestamp.Logical = 0 require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, &laterRec)) - require.NoError(t, ptv.Verify(ctx, laterRec.ID.GetUUID())) + require.NoError( + t, + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, laterRec.Timestamp, laterRec.DeprecatedSpans), + ) // Release the record that had succeeded and ensure that GC eventually // happens up to the protected timestamp of the new record. @@ -263,3 +270,29 @@ func TestProtectedTimestamps(t *testing.T) { require.Len(t, state.Records, 0) require.Equal(t, int(state.NumRecords), len(state.Records)) } + +func verifyProtectionTimestampExistsOnSpans( + ctx context.Context, + tc *testcluster.TestCluster, + provider protectedts.Provider, + pts hlc.Timestamp, + spans roachpb.Spans, +) error { + if err := provider.Refresh(ctx, tc.Server(0).Clock().Now()); err != nil { + return err + } + for _, sp := range spans { + timestamps, _ := provider.GetProtectionTimestamps(ctx, sp) + found := false + for _, ts := range timestamps { + if ts.Equal(pts) { + found = true + break + } + } + if !found { + return errors.Newf("protection timestamp %s does not exist on span %s", pts, sp) + } + } + return nil +} diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index a14aa81a3a3c..3ead2166f4be 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -3579,6 +3579,17 @@ func TestStrictGCEnforcement(t *testing.T) { r.ReadProtectedTimestamps(ctx) } } + refreshCacheAndUpdatePTSState = func(t *testing.T, nodeID roachpb.NodeID) { + for i := 0; i < tc.NumServers(); i++ { + if tc.Server(i).NodeID() != nodeID { + continue + } + ptp := tc.Server(i).ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider + require.NoError(t, ptp.Refresh(ctx, tc.Server(i).Clock().Now())) + _, r := getFirstStoreReplica(t, tc.Server(i), tableKey) + r.ReadProtectedTimestamps(ctx) + } + } ) { @@ -3632,13 +3643,15 @@ func TestStrictGCEnforcement(t *testing.T) { })) assertScanRejected(t) - require.NoError(t, ptp.Verify(ctx, rec.ID.GetUUID())) + desc, err := tc.LookupRange(tableKey) + require.NoError(t, err) + target, err := tc.FindRangeLeaseHolder(desc, nil) + require.NoError(t, err) + refreshCacheAndUpdatePTSState(t, target.NodeID) assertScanOk(t) // Transfer the lease and demonstrate that the query succeeds because we're // cautious in the face of lease transfers. - desc, err := tc.LookupRange(tableKey) - require.NoError(t, err) require.NoError(t, tc.TransferRangeLease(desc, tc.Target(1))) assertScanOk(t) }) diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 6ed71ebcebb6..febfab7a0c43 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -3358,34 +3358,22 @@ func (r *Replica) adminScatter( }, nil } +// TODO(arul): AdminVerifyProtectedTimestampRequest can entirely go away in +// 22.2. func (r *Replica) adminVerifyProtectedTimestamp( - ctx context.Context, args roachpb.AdminVerifyProtectedTimestampRequest, + ctx context.Context, _ roachpb.AdminVerifyProtectedTimestampRequest, ) (resp roachpb.AdminVerifyProtectedTimestampResponse, err error) { - var doesNotApplyReason string - resp.Verified, doesNotApplyReason, err = r.protectedTimestampRecordApplies(ctx, &args) - if err != nil { - return resp, err - } - - // In certain cases we do not want to return an error even if we failed to - // verify the protected ts record. This ensures that executeAdminBatch adds - // the response to the batch, thereby allowing us to aggregate the - // verification failures across all AdminVerifyProtectedTimestampRequests and - // construct a more informative error to show to the user. - if doesNotApplyReason != "" { - if !resp.Verified { - desc := r.Desc() - failedRange := roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{ - RangeID: int64(desc.GetRangeID()), - StartKey: desc.GetStartKey(), - EndKey: desc.EndKey, - Reason: doesNotApplyReason, - } - resp.VerificationFailedRanges = append(resp.VerificationFailedRanges, failedRange) - // TODO(adityamaru): This is here for compatibility with 20.2, remove in - // 21.2. - resp.DeprecatedFailedRanges = append(resp.DeprecatedFailedRanges, *r.Desc()) - } - } + // AdminVerifyProtectedTimestampRequest is not supported starting from the + // 22.1 release. We expect nodes running a 22.1 binary to still service this + // request in a {21.2, 22.1} mixed version cluster. This can happen if the + // request is initiated on a 21.2 node and the leaseholder of the range it is + // trying to verify is on a 22.1 node. + // + // We simply return true without attempting to verify in such a case. This + // ensures upstream jobs (backups) don't fail as a result. It is okay to + // return true regardless even if the PTS record being verified does not apply + // as the failure mode is non-destructive. Infact, this is the reason we're + // no longer supporting Verification past 22.1. + resp.Verified = true return resp, nil } diff --git a/pkg/kv/kvserver/replica_protected_timestamp.go b/pkg/kv/kvserver/replica_protected_timestamp.go index cc23a79c783a..efab180fdc65 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp.go +++ b/pkg/kv/kvserver/replica_protected_timestamp.go @@ -12,11 +12,9 @@ package kvserver import ( "context" - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -67,41 +65,6 @@ func (r *Replica) maybeUpdateCachedProtectedTS(ts *cachedProtectedTimestampState } } -// protectedTimestampRecordApplies returns true if it is this case that the -// record which protects the `protected` timestamp. It returns false if it may -// not. If the state of the cache is not sufficiently new to determine whether -// the record will apply, the cache is refreshed and then the check is performed -// again. See r.protectedTimestampRecordCurrentlyApplies() for more details. -func (r *Replica) protectedTimestampRecordApplies( - ctx context.Context, args *roachpb.AdminVerifyProtectedTimestampRequest, -) (willApply bool, doesNotApplyReason string, _ error) { - // Check the state of the cache without a refresh. - willApply, cacheTooOld, doesNotApplyReason, err := r.protectedTimestampRecordCurrentlyApplies( - ctx, args) - if err != nil { - return false, doesNotApplyReason, err - } - if !cacheTooOld { - return willApply, doesNotApplyReason, nil - } - // Refresh the cache so that we know that the next time we come around we're - // certain to either see the record or see a timestamp for readAt that is - // greater than or equal to recordAliveAt. - if err := r.store.protectedtsCache.Refresh(ctx, args.RecordAliveAt); err != nil { - return false, doesNotApplyReason, err - } - willApply, cacheTooOld, doesNotApplyReason, err = r.protectedTimestampRecordCurrentlyApplies( - ctx, args) - if err != nil { - return false, doesNotApplyReason, err - } - if cacheTooOld { - return false, doesNotApplyReason, errors.AssertionFailedf( - "cache was not updated after being refreshed") - } - return willApply, doesNotApplyReason, nil -} - func (r *Replica) readProtectedTimestampsRLocked( ctx context.Context, f func(r *ptpb.Record), ) (ts cachedProtectedTimestampState) { @@ -131,123 +94,6 @@ func (r *Replica) readProtectedTimestampsRLocked( return ts } -// protectedTimestampRecordCurrentlyApplies determines whether a record with -// the specified ID which protects `protected` and is known to exist at -// `recordAliveAt` will apply given the current state of the cache. This method -// is called by `r.protectedTimestampRecordApplies()`. It may be the case that -// the current state of the cache is too old to determine whether the record -// will apply. In such cases the cache should be refreshed to recordAliveAt and -// then this method should be called again. -// In certain cases we return a doesNotApplyReason explaining why the protected -// ts record does not currently apply. We do not want to return an error so that -// we can aggregate the reasons across multiple -// AdminVerifyProtectedTimestampRequest, as explained in -// adminVerifyProtectedTimestamp. -func (r *Replica) protectedTimestampRecordCurrentlyApplies( - ctx context.Context, args *roachpb.AdminVerifyProtectedTimestampRequest, -) (willApply, cacheTooOld bool, doesNotApplyReason string, _ error) { - // We first need to check that we're the current leaseholder. - // TODO(ajwerner): what other conditions with regards to time do we need to - // check? I don't think there are any. If the recordAliveAt is after our - // liveness expiration that's okay because we're either going to find the - // record or we're not and if we don't then we'll push the cache and re-assert - // that we're still the leaseholder. If somebody else becomes the leaseholder - // then they will have to go through the same process. - ls, pErr := r.redirectOnOrAcquireLease(ctx) - if pErr != nil { - return false, false, "", pErr.GoError() - } - - // NB: It should be the case that the recordAliveAt timestamp - // is before the current time and that the above lease check means that - // the replica is the leaseholder at the current time. If recordAliveAt - // happened to be newer than the current time we'd need to make sure that - // the current Replica will be live at that time. Given that recordAliveAt - // has to be before the batch timestamp for this request and we should - // have forwarded the local clock to the batch timestamp this can't - // happen. - // TODO(ajwerner): do we need to assert that indeed the recordAliveAt precedes - // the batch timestamp? Probably not a bad sanity check. - - // We may be reading the protected timestamp cache while we're holding - // the Replica.mu for reading. If we do so and find newer state in the cache - // then we want to, update the replica's cache of its state. The guarantee - // we provide is that if a record is successfully verified then the Replica's - // cachedProtectedTS will have a readAt value high enough to include that - // record. - var read cachedProtectedTimestampState - defer r.maybeUpdateCachedProtectedTS(&read) - r.mu.RLock() - defer r.mu.RUnlock() - defer read.clearIfNotNewer(r.mu.cachedProtectedTS) - - // If the key that routed this request to this range is now out of this - // range's bounds, return an error for the client to try again on the - // correct range. - desc := r.descRLocked() - if !kvserverbase.ContainsKeyRange(desc, args.Key, args.EndKey) { - return false, false, "", roachpb.NewRangeKeyMismatchErrorWithCTPolicy(ctx, args.Key, args.EndKey, desc, - r.mu.state.Lease, r.closedTimestampPolicyRLocked()) - } - if args.Protected.LessEq(*r.mu.state.GCThreshold) { - gcReason := fmt.Sprintf("protected ts: %s is less than equal to the GCThreshold: %s for the"+ - " range %s - %s", args.Protected.String(), r.mu.state.GCThreshold.String(), - desc.StartKey.String(), desc.EndKey.String()) - return false, false, gcReason, nil - } - if args.RecordAliveAt.Less(ls.Lease.Start.ToTimestamp()) { - return true, false, "", nil - } - - // Now we're in the case where maybe it is possible that we're going to later - // attempt to set the GC threshold above our protected point so to prevent - // that we add some state to the replica. - r.protectedTimestampMu.Lock() - defer r.protectedTimestampMu.Unlock() - if args.Protected.Less(r.protectedTimestampMu.pendingGCThreshold) { - gcReason := fmt.Sprintf( - "protected ts: %s is less than the pending GCThreshold: %s for the range %s - %s", - args.Protected.String(), r.protectedTimestampMu.pendingGCThreshold.String(), - desc.StartKey.String(), desc.EndKey.String()) - return false, false, gcReason, nil - } - - var seen bool - read = r.readProtectedTimestampsRLocked(ctx, func(r *ptpb.Record) { - // Comparing record ID and the timestamp ensures that we find the record - // that we are verifying. - // A PTS record can be updated with a new Timestamp to protect, and so we - // need to ensure that we are not seeing the old version of the record in - // case the cache has not been updated. - if r.ID.GetUUID() == args.RecordID && args.Protected.LessEq(r.Timestamp) { - seen = true - } - }) - - // If we observed the record in question then we know that all future attempts - // to run GC will observe the Record if it still exists. The one hazard we - // need to avoid is a race whereby an attempt to run GC first checks the - // protected timestamp state and then attempts to increase the GC threshold. - // We set the minStateReadTimestamp here to avoid such races. The MVCC GC - // queue will call markPendingGC just prior to sending a request to update the - // GC threshold which will verify the safety of the new value relative to - // minStateReadTimestamp. - if seen { - r.protectedTimestampMu.minStateReadTimestamp = read.readAt - return true, false, "", nil - } - - isCacheTooOld := read.readAt.Less(args.RecordAliveAt) - // Protected timestamp state has progressed past the point at which we - // should see this record. This implies that the record has been removed. - if !isCacheTooOld { - recordRemovedReason := "protected ts record has been removed" - return false, false, recordRemovedReason, nil - } - // Retry, since the cache is too old. - return false, true, "", nil -} - // checkProtectedTimestampsForGC determines whether the Replica can run GC. If // the Replica can run GC, this method returns the latest timestamp which can be // used to determine a valid new GCThreshold. The policy is passed in rather diff --git a/pkg/kv/kvserver/replica_protected_timestamp_test.go b/pkg/kv/kvserver/replica_protected_timestamp_test.go index 31c54420f8a0..a70be6270156 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp_test.go +++ b/pkg/kv/kvserver/replica_protected_timestamp_test.go @@ -25,380 +25,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) -// TestProtectedTimestampRecordApplies exercises -// Replica.protectedTimestampWillApply() at a low level. -// It does so by passing a Replica connected to an already -// shut down store to a variety of test cases. -func TestProtectedTimestampRecordApplies(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - ctx := context.Background() - - makeArgs := func(r *Replica, ts, aliveAt hlc.Timestamp) roachpb.AdminVerifyProtectedTimestampRequest { - args := roachpb.AdminVerifyProtectedTimestampRequest{ - RecordID: uuid.MakeV4(), - Protected: ts, - RecordAliveAt: aliveAt, - } - args.Key, args.EndKey = r.Desc().StartKey.AsRawKey(), r.Desc().EndKey.AsRawKey() - return args - } - for _, testCase := range []struct { - name string - // Note that the store underneath the passed in Replica has been stopped. - // This leaves the test to mutate the Replica state as it sees fit. - test func(t *testing.T, r *Replica, mt *manualCache) - }{ - - // Test that if the lease started after the timestamp at which the record - // was known to be live then we know that the Replica cannot GC until it - // reads protected timestamp state after the lease start time. If the - // relevant record is not found then it must have been removed. - { - name: "lease started after", - test: func(t *testing.T, r *Replica, mt *manualCache) { - r.mu.state.Lease.Start = r.store.Clock().NowAsClockTimestamp() - l, _ := r.GetLease() - aliveAt := l.Start.ToTimestamp().Prev() - ts := aliveAt.Prev() - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReaason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.True(t, willApply) - require.NoError(t, err) - require.Empty(t, doesNotApplyReaason) - }, - }, - // If the GC threshold is already newer than the timestamp we want to - // protect then we failed. - { - name: "gc threshold is after ts", - test: func(t *testing.T, r *Replica, mt *manualCache) { - thresh := r.store.Clock().Now() - r.mu.state.GCThreshold = &thresh - ts := thresh.Prev().Prev() - aliveAt := ts.Next() - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.NoError(t, err) - require.Regexp(t, fmt.Sprintf("protected ts: %s is less than equal to the GCThreshold: %s"+ - " for the range /Min - /Max", ts.String(), thresh.String()), doesNotApplyReason) - }, - }, - // If the GC threshold we're about to protect is newer than the timestamp - // we want to protect then we're almost certain to fail. Treat it as a - // failure. - { - name: "pending GC threshold is newer than the timestamp we want to protect", - test: func(t *testing.T, r *Replica, mt *manualCache) { - thresh := r.store.Clock().Now() - require.NoError(t, r.markPendingGC(hlc.Timestamp{}, thresh)) - ts := thresh.Prev().Prev() - aliveAt := ts.Next() - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.NoError(t, err) - require.Regexp(t, fmt.Sprintf("protected ts: %s is less than the pending GCThreshold: %s"+ - " for the range /Min - /Max", ts.String(), thresh.String()), doesNotApplyReason) - }, - }, - // If the timestamp at which the record is known to be alive is newer than - // our current view of the protected timestamp subsystem and we don't - // already see the record, then we will refresh. In this case we refresh - // and find it. We also verify that we cannot set the pending gc threshold - // to above the timestamp we put in the record. - { - name: "newer aliveAt triggers refresh leading to success", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = ts.Prev() - args := makeArgs(r, ts, aliveAt) - mt.refresh = func(_ context.Context, refreshTo hlc.Timestamp) error { - require.Equal(t, refreshTo, aliveAt) - mt.records = append(mt.records, &ptpb.Record{ - ID: args.RecordID.GetBytes(), - Timestamp: ts, - DeprecatedSpans: []roachpb.Span{ - { - Key: roachpb.Key(r.Desc().StartKey), - EndKey: roachpb.Key(r.Desc().StartKey.Next()), - }, - }, - }) - mt.asOf = refreshTo.Next() - return nil - } - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.True(t, willApply) - require.NoError(t, err) - require.Empty(t, doesNotApplyReason) - require.Equal(t, - fmt.Sprintf("cannot set gc threshold to %v because read at %v < min %v", - ts.Next(), ts, aliveAt.Next()), - r.markPendingGC(ts, ts.Next()).Error()) - }, - }, - // If the timestamp of a record is updated then a verify request must see - // the correct version of the record. If the request finds the version of - // the record with the older timestamp then it must refresh the cache and - // attempt to verify again. - { - name: "find correct record on timestamp update", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - mt.asOf = ts.Prev() - id := uuid.MakeV4() - // Insert a record. - oldTimestamp := ts - mt.records = append(mt.records, &ptpb.Record{ - ID: id.GetBytes(), - Timestamp: oldTimestamp, - DeprecatedSpans: []roachpb.Span{ - { - Key: roachpb.Key(r.Desc().StartKey), - EndKey: roachpb.Key(r.Desc().StartKey.Next()), - }, - }, - }) - // Assume the record has an updated timestamp that we are trying to - // verify. - updatedTimestamp := ts.Next() - aliveAt := ts.Next().Next() - args := makeArgs(r, updatedTimestamp, aliveAt) - args.RecordID = id - - var cacheIsRefreshed bool - mt.refresh = func(_ context.Context, refreshTo hlc.Timestamp) error { - cacheIsRefreshed = true - require.Equal(t, refreshTo, aliveAt) - // Update the record timestamp so that post cache refresh we see the - // record with the updated timestamp. - mt.records[0].Timestamp = updatedTimestamp - mt.asOf = refreshTo.Next() - return nil - } - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.True(t, cacheIsRefreshed) - require.True(t, willApply) - require.NoError(t, err) - require.Empty(t, doesNotApplyReason) - }, - }, - { - name: "find correct record on multiple timestamp update", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - mt.asOf = ts.Prev() - id := uuid.MakeV4() - // Insert a record. - oldTimestamp := ts - mt.records = append(mt.records, &ptpb.Record{ - ID: id.GetBytes(), - Timestamp: oldTimestamp, - DeprecatedSpans: []roachpb.Span{ - { - Key: roachpb.Key(r.Desc().StartKey), - EndKey: roachpb.Key(r.Desc().StartKey.Next()), - }, - }, - }) - // Assume the record has an updated timestamp that we are trying to - // verify. - updatedTimestamp := ts.Next() - aliveAt := ts.Next().Next() - args := makeArgs(r, updatedTimestamp, aliveAt) - args.RecordID = id - - var cacheIsRefreshed bool - mt.refresh = func(_ context.Context, refreshTo hlc.Timestamp) error { - cacheIsRefreshed = true - require.Equal(t, refreshTo, aliveAt) - // Assume that there was a second timestamp update while the - // verification for the first was inflight. - // Verification of the first update should still pass since the cache - // sees a record with a later timestamp than the one we are verifying. - anotherUpdateTimestamp := updatedTimestamp.Next() - mt.records[0].Timestamp = anotherUpdateTimestamp - mt.asOf = refreshTo.Next() - return nil - } - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.True(t, cacheIsRefreshed) - require.True(t, willApply) - require.NoError(t, err) - require.Empty(t, doesNotApplyReason) - }, - }, - // If the timestamp at which the record is known to be alive is older than - // our current view of the protected timestamp subsystem and we don't - // already see the record, then we know that the record must have been - // deleted already. Ensure we fail. - { - name: "record does not exist", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = aliveAt.Next() - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.NoError(t, err) - require.Regexp(t, "protected ts record has been removed", doesNotApplyReason) - }, - }, - // If we see the record then we know we're good. - { - name: "record already exists", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - args := makeArgs(r, ts, aliveAt) - mt.asOf = aliveAt.Next() - mt.records = append(mt.records, &ptpb.Record{ - ID: args.RecordID.GetBytes(), - Timestamp: ts, - DeprecatedSpans: []roachpb.Span{ - { - Key: keys.MinKey, - EndKey: keys.MaxKey, - }, - }, - }) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.True(t, willApply) - require.NoError(t, err) - require.Empty(t, doesNotApplyReason) - }, - }, - // Ensure that a failure to Refresh propagates. - { - name: "refresh fails", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = ts.Prev() - mt.refresh = func(_ context.Context, refreshTo hlc.Timestamp) error { - return errors.New("boom") - } - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.EqualError(t, err, "boom") - require.Empty(t, doesNotApplyReason) - }, - }, - // Ensure NLE propagates. - { - name: "not leaseholder before refresh", - test: func(t *testing.T, r *Replica, mt *manualCache) { - r.mu.Lock() - lease := *r.mu.state.Lease - lease.Sequence++ - lease.Replica = roachpb.ReplicaDescriptor{ - ReplicaID: 2, - StoreID: 2, - NodeID: 2, - } - r.mu.state.Lease = &lease - r.mu.Unlock() - ts := r.store.Clock().Now() - aliveAt := ts.Prev().Prev() - mt.asOf = ts.Prev() - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.Error(t, err) - require.Regexp(t, "NotLeaseHolderError", err.Error()) - require.Empty(t, doesNotApplyReason) - }, - }, - // Ensure NLE after performing a refresh propagates. - { - name: "not leaseholder after refresh", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = ts.Prev() - mt.refresh = func(ctx context.Context, refreshTo hlc.Timestamp) error { - r.mu.Lock() - defer r.mu.Unlock() - lease := *r.mu.state.Lease - lease.Sequence++ - lease.Replica = roachpb.ReplicaDescriptor{ - ReplicaID: 2, - StoreID: 2, - NodeID: 2, - } - r.mu.state.Lease = &lease - return nil - } - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.Error(t, err) - require.Regexp(t, "NotLeaseHolderError", err.Error()) - require.Empty(t, doesNotApplyReason) - }, - }, - // If refresh succeeds but the timestamp of the cache does not advance as - // anticipated, ensure that an assertion failure error is returned. - { - name: "successful refresh does not update timestamp (assertion failure)", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = ts.Prev() - mt.refresh = func(_ context.Context, refreshTo hlc.Timestamp) error { - return nil - } - args := makeArgs(r, ts, aliveAt) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.EqualError(t, err, "cache was not updated after being refreshed") - require.True(t, errors.IsAssertionFailure(err), "%v", err) - require.Empty(t, doesNotApplyReason) - }, - }, - // If a request header is for a key span which is not owned by this replica, - // ensure that a roachpb.RangeKeyMismatchError is returned. - { - name: "request span is respected", - test: func(t *testing.T, r *Replica, mt *manualCache) { - ts := r.store.Clock().Now() - aliveAt := ts.Next() - mt.asOf = ts.Prev() - args := makeArgs(r, ts, aliveAt) - r.mu.state.Desc.StartKey = roachpb.RKey(keys.TableDataMax) - willApply, doesNotApplyReason, err := r.protectedTimestampRecordApplies(ctx, &args) - require.False(t, willApply) - require.Error(t, err) - require.Regexp(t, "key range /Min-/Max outside of bounds of range /Table/Max-/Max", err.Error()) - require.Empty(t, doesNotApplyReason) - }, - }, - } { - t.Run(testCase.name, func(t *testing.T) { - tc := testContext{} - tsc := TestStoreConfig(nil) - mc := &manualCache{} - tsc.ProtectedTimestampCache = mc - // Under extreme stressrace scenarios the single replica can somehow - // lose the lease. Make the timeout extremely long. - tsc.RaftConfig.RangeLeaseRaftElectionTimeoutMultiplier = 100 - stopper := stop.NewStopper() - tc.StartWithStoreConfig(ctx, t, stopper, tsc) - stopper.Stop(ctx) - testCase.test(t, tc.repl, mc) - }) - } -} - // TestCheckProtectedTimestampsForGC exercises // Replica.checkProtectedTimestampsForGC() at a low level. // It does so by passing a Replica connected to an already From 9fbf507f3eae0a37ad792288bd9878d1f31c6d93 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Fri, 25 Feb 2022 17:00:52 -0500 Subject: [PATCH 2/6] spanconfigkvsubscriber,kvserver: fix KVSubscriber bug We had a bug in the KVSubscriber where we were invoking a copy of the handler instead of the handler stored. This meant that we'd never treat handlers as "initialized". As a result, we would always invoke them with the everything span, and as a result, visit all replicas on the stores in reaction to span config updates. See datadriven test diffs for an illustration. Fixing the above lead to unearthing an interesting bug in how we were deciding to enqueue replicas in the split queue. Previously, if we received a span config update that implied a split and the update corresponded to the right-hand side post split, we would skip enqueuing the replica in the split queue. The assumption was that we'd get an update corresponding to the LHS of the split for the same replica and that update would enqueue the replica. This doesn't always hold true though. For example, consider the case when a new table is created and must be split from its (left) adjacent table's range. This only results in a single update, corresponding to the new table's span, which is the right-hand side post split. This patch moves to nudging the split queue for all updates, not just left-hand side updates, for the reason above. Release note: None Release justification: bug fixes in new functionality --- pkg/kv/kvserver/store.go | 70 +++++++++---------- .../spanconfigkvsubscriber/kvsubscriber.go | 8 ++- .../spanconfigkvsubscriber/testdata/basic | 1 - .../testdata/buffer_overflow | 1 - .../testdata/system_span_configs | 2 - .../system_span_configs_secondary_tenants | 2 - 6 files changed, 39 insertions(+), 45 deletions(-) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index cb2c980c64e7..a368853dd4e1 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2395,45 +2395,43 @@ func (s *Store) onSpanConfigUpdate(ctx context.Context, updated roachpb.Span) { return nil // placeholder; ignore } + replCtx := repl.AnnotateCtx(ctx) startKey := repl.Desc().StartKey - if !sp.ContainsKey(startKey) { - // It's possible that the update we're receiving here is the - // right-hand side of a span config getting split. Think of - // installing a zone config on some partition of an index where - // previously there was none on any of the partitions. The range - // spanning the entire index would have to split on the - // partition boundary, and before it does so, it's possible that - // it would receive a span config update for just the partition. + if sp.ContainsKey(startKey) { + // It's possible that the update we're receiving here implies a split. + // If the update corresponds to what would be the config for the + // right-hand side after the split, we avoid clobbering the pre-split + // range's embedded span config by checking if the start key is part of + // the update. // - // To avoid clobbering the pre-split range's embedded span - // config with the partition's config, we'll ensure that the - // range's start key is part of the update. We don't have to - // enqueue the range in the split queue here, that takes place - // when processing the left-hand side span config update. - - return nil // ignore - } - - // TODO(irfansharif): It's possible for a config to be applied over an - // entire range when it only pertains to the first half of the range. - // This will be corrected shortly -- we enqueue the range for a split - // below where we then apply the right config on each half. But still, - // it's surprising behavior and gets in the way of a desirable - // consistency guarantee: a key's config at any point in time is one - // that was explicitly declared over it, or the default config. - // - // We can do better, we can skip applying the config entirely and - // enqueue the split, then relying on the split trigger to install - // the right configs on each half. The current structure is as it is - // to maintain parity with the system config span variant. - - replCtx := repl.AnnotateCtx(ctx) - conf, err := s.cfg.SpanConfigSubscriber.GetSpanConfigForKey(replCtx, startKey) - if err != nil { - log.Errorf(ctx, "skipped applying update, unexpected error reading from subscriber: %v", err) - return err + // Even if we're dealing with what would be the right-hand side after + // the split is processed, we still want to nudge the split queue + // below -- we can't instead rely on there being an update for the + // left-hand side of the split. Concretely, consider the case when a + // new table is added with a different configuration to its (left) + // adjacent table. This results in a single update, corresponding to the + // new table's span, which forms the right-hand side post split. + + // TODO(irfansharif): It's possible for a config to be applied over an + // entire range when it only pertains to the first half of the range. + // This will be corrected shortly -- we enqueue the range for a split + // below where we then apply the right config on each half. But still, + // it's surprising behavior and gets in the way of a desirable + // consistency guarantee: a key's config at any point in time is one + // that was explicitly declared over it, or the default config. + // + // We can do better, we can skip applying the config entirely and + // enqueue the split, then relying on the split trigger to install + // the right configs on each half. The current structure is as it is + // to maintain parity with the system config span variant. + + conf, err := s.cfg.SpanConfigSubscriber.GetSpanConfigForKey(replCtx, startKey) + if err != nil { + log.Errorf(ctx, "skipped applying update, unexpected error reading from subscriber: %v", err) + return err + } + repl.SetSpanConfig(conf) } - repl.SetSpanConfig(conf) // TODO(irfansharif): For symmetry with the system config span variant, // we queue blindly; we could instead only queue it if we knew the diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go index 84ca69e3eaf7..0714c2bdbcec 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go @@ -229,8 +229,9 @@ func (s *KVSubscriber) handleCompleteUpdate( s.mu.lastUpdated = ts handlers := s.mu.handlers s.mu.Unlock() - for _, h := range handlers { - h.invoke(ctx, keys.EverythingSpan) + for i := range handlers { + handler := &handlers[i] // mutated by invoke + handler.invoke(ctx, keys.EverythingSpan) } } @@ -248,7 +249,8 @@ func (s *KVSubscriber) handlePartialUpdate( handlers := s.mu.handlers s.mu.Unlock() - for _, handler := range handlers { + for i := range handlers { + handler := &handlers[i] // mutated by invoke for _, ev := range events { target := ev.(*bufferEvent).Update.Target handler.invoke(ctx, target.KeyspaceTargeted()) diff --git a/pkg/spanconfig/spanconfigkvsubscriber/testdata/basic b/pkg/spanconfig/spanconfigkvsubscriber/testdata/basic index d188e8dcaeed..54d052c09945 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/testdata/basic +++ b/pkg/spanconfig/spanconfigkvsubscriber/testdata/basic @@ -40,7 +40,6 @@ delete [d,f) updates ---- -[/Min,/Max) [d,f) store-reader key=d diff --git a/pkg/spanconfig/spanconfigkvsubscriber/testdata/buffer_overflow b/pkg/spanconfig/spanconfigkvsubscriber/testdata/buffer_overflow index 7a1b0d5397c4..997caa397365 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/testdata/buffer_overflow +++ b/pkg/spanconfig/spanconfigkvsubscriber/testdata/buffer_overflow @@ -75,7 +75,6 @@ upsert [a,c):C updates ---- -[/Min,/Max) [a,c) store-reader key=a diff --git a/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs b/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs index d345d8533e8d..200a7c790b76 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs +++ b/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs @@ -55,7 +55,6 @@ upsert {source=1, target=1}:Y # [/Min,/Tenant/2] corresponds to the system tenant targeting itself. updates ---- -[/Min,/Max) [/Min,/Tenant/2) # Ensure configs are correctly hydrated when we read them. @@ -93,7 +92,6 @@ upsert {source=1, target=1}:V updates ---- -[/Min,/Max) [/Min,/Tenant/2) store-reader key=a diff --git a/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs_secondary_tenants b/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs_secondary_tenants index 14fe1bfd8433..708e288b768d 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs_secondary_tenants +++ b/pkg/spanconfig/spanconfigkvsubscriber/testdata/system_span_configs_secondary_tenants @@ -21,7 +21,6 @@ delete {source=1,target=20} updates ---- -[/Min,/Max) [/Tenant/20,/Tenant/21) # Lastly, update a system span config set on a secondary tenant's keyspace and @@ -32,5 +31,4 @@ upsert {source=10, target=10}:U updates ---- -[/Min,/Max) [/Tenant/10,/Tenant/11) From 365521a9d9a263c226948b570ca844d2371bb3b2 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Fri, 21 Jan 2022 12:58:20 -0500 Subject: [PATCH 3/6] spanconfig,protectedts: introduce the `ProtectedTSReader` interface This patch introduces the `spanconfig.ProtectedTSReader` interface. It is inteded to replace the `protectedts.Cache` as KVs integration point for querying protectedTS metadata in the v2 of the protectedTS subsystem. For now, the `protectedts.Cache` is the only implementor of it. We will have the `KVSubscriber` implement it as well once we start shipping protected timestamp information on span configurations. The `ProtectedTSReader` interface is also inteded to serve as an adapter interface between v1 and v2 of the PTS subsystem. In particular, for v21.2, we will consult both the `protectedts.Cache` and `KVSubscriber` for PTS information. The logic here will be gated behind a call to `GetProtectionTimetamps`, which is the only method this interface provides. Release note: None Release justification: low risk, high benefit changes to existing functionality. --- pkg/kv/kvserver/BUILD.bazel | 4 +- pkg/kv/kvserver/client_protectedts_test.go | 29 ++- pkg/kv/kvserver/client_replica_test.go | 15 +- pkg/kv/kvserver/client_split_test.go | 8 +- pkg/kv/kvserver/helpers_test.go | 2 +- .../kvserver/protectedts/ptcache/BUILD.bazel | 1 + pkg/kv/kvserver/protectedts/ptcache/cache.go | 17 ++ .../protectedts/ptcache/cache_test.go | 122 ++++++++-- .../protectedts/ptprovider/provider.go | 25 ++- pkg/kv/kvserver/replica.go | 4 +- .../kvserver/replica_protected_timestamp.go | 57 +++-- .../replica_protected_timestamp_test.go | 209 ++++++++++-------- pkg/kv/kvserver/store.go | 14 +- pkg/server/BUILD.bazel | 2 + pkg/server/server.go | 59 ++--- pkg/spanconfig/spanconfig.go | 28 +++ .../spanconfigptsreader/BUILD.bazel | 15 ++ pkg/spanconfig/spanconfigptsreader/adapter.go | 78 +++++++ 18 files changed, 486 insertions(+), 203 deletions(-) create mode 100644 pkg/spanconfig/spanconfigptsreader/BUILD.bazel create mode 100644 pkg/spanconfig/spanconfigptsreader/adapter.go diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index ecf5ab794434..fca39bff411c 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -132,8 +132,6 @@ go_library( "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", - "//pkg/kv/kvserver/protectedts", - "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/kv/kvserver/raftentry", "//pkg/kv/kvserver/rangefeed", "//pkg/kv/kvserver/rditer", @@ -336,7 +334,6 @@ go_test( "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", - "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/kv/kvserver/protectedts/ptstorage", "//pkg/kv/kvserver/raftentry", @@ -359,6 +356,7 @@ go_test( "//pkg/server/telemetry", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigstore", "//pkg/sql", "//pkg/sql/catalog/bootstrap", diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index ce55646c9696..6622d7ff993a 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -18,10 +18,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -212,10 +213,10 @@ func TestProtectedTimestamps(t *testing.T) { // Verify that the record did indeed make its way down into KV where the // replica can read it from. - ptp := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider + ptsReader := tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().ProtectedTimestampReader require.NoError( t, - verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, ptsRec.Timestamp, ptsRec.DeprecatedSpans), + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptsReader, ptsRec.Timestamp, ptsRec.DeprecatedSpans), ) // Make a new record that is doomed to fail. @@ -227,11 +228,12 @@ func TestProtectedTimestamps(t *testing.T) { _, err = ptsWithDB.GetRecord(ctx, nil /* txn */, failedRec.ID.GetUUID()) require.NoError(t, err) + // Verify that the record did indeed make its way down into KV where the // replica can read it from. We then verify (below) that the failed record // does not affect the ability to GC. require.NoError( t, - verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, failedRec.Timestamp, failedRec.DeprecatedSpans), + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptsReader, failedRec.Timestamp, failedRec.DeprecatedSpans), ) // Add a new record that is after the old record. @@ -242,7 +244,7 @@ func TestProtectedTimestamps(t *testing.T) { require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, &laterRec)) require.NoError( t, - verifyProtectionTimestampExistsOnSpans(ctx, tc, ptp, laterRec.Timestamp, laterRec.DeprecatedSpans), + verifyProtectionTimestampExistsOnSpans(ctx, tc, ptsReader, laterRec.Timestamp, laterRec.DeprecatedSpans), ) // Release the record that had succeeded and ensure that GC eventually @@ -271,27 +273,32 @@ func TestProtectedTimestamps(t *testing.T) { require.Equal(t, int(state.NumRecords), len(state.Records)) } +// verifyProtectionTimestampExistsOnSpans refreshes the PTS state in KV and +// ensures a protection at the given protectionTimestamp exists for all the +// supplied spans. func verifyProtectionTimestampExistsOnSpans( ctx context.Context, tc *testcluster.TestCluster, - provider protectedts.Provider, - pts hlc.Timestamp, + ptsReader spanconfig.ProtectedTSReader, + protectionTimestamp hlc.Timestamp, spans roachpb.Spans, ) error { - if err := provider.Refresh(ctx, tc.Server(0).Clock().Now()); err != nil { + if err := spanconfigptsreader.TestingRefreshPTSState( + ctx, ptsReader, tc.Server(0).Clock().Now(), + ); err != nil { return err } for _, sp := range spans { - timestamps, _ := provider.GetProtectionTimestamps(ctx, sp) + timestamps, _ := ptsReader.GetProtectionTimestamps(ctx, sp) found := false for _, ts := range timestamps { - if ts.Equal(pts) { + if ts.Equal(protectionTimestamp) { found = true break } } if !found { - return errors.Newf("protection timestamp %s does not exist on span %s", pts, sp) + return errors.Newf("protection timestamp %s does not exist on span %s", protectionTimestamp, sp) } } return nil diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 3ead2166f4be..427f011593a6 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/storage" @@ -3517,10 +3518,9 @@ func TestStrictGCEnforcement(t *testing.T) { t.Helper() testutils.SucceedsSoon(t, func() error { for i := 0; i < tc.NumServers(); i++ { - ptp := tc.Server(i).ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider - if ptp.Iterate(ctx, tableKey, tableKey, func(record *ptpb.Record) (wantMore bool) { - return false - }).Less(min) { + ptsReader := tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().ProtectedTimestampReader + _, asOf := ptsReader.GetProtectionTimestamps(ctx, tableSpan) + if asOf.Less(min) { return errors.Errorf("not yet read") } } @@ -3572,10 +3572,13 @@ func TestStrictGCEnforcement(t *testing.T) { } refreshPastLeaseStart = func(t *testing.T) { for i := 0; i < tc.NumServers(); i++ { - ptp := tc.Server(i).ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider + ptsReader := tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().ProtectedTimestampReader _, r := getFirstStoreReplica(t, tc.Server(i), tableKey) l, _ := r.GetLease() - require.NoError(t, ptp.Refresh(ctx, l.Start.ToTimestamp().Next())) + require.NoError( + t, + spanconfigptsreader.TestingRefreshPTSState(ctx, ptsReader, l.Start.ToTimestamp().Next()), + ) r.ReadProtectedTimestamps(ctx) } } diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 1847bf028ae5..2957aba56cbd 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/storage" @@ -2431,7 +2432,12 @@ func TestUnsplittableRange(t *testing.T) { // row. Once the first version of the row is cleaned up, the range should // exit the split queue purgatory. We need to tickle the protected timestamp // subsystem to release a timestamp at which we get to actually remove the data. - require.NoError(t, store.GetStoreConfig().ProtectedTimestampCache.Refresh(ctx, s.Clock().Now())) + require.NoError( + t, + spanconfigptsreader.TestingRefreshPTSState( + ctx, store.GetStoreConfig().ProtectedTimestampReader, s.Clock().Now(), + ), + ) repl := store.LookupReplica(tableKey) if err := store.ManualMVCCGC(repl); err != nil { t.Fatal(err) diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index a6e92183026d..916ce3afc2c8 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -488,7 +488,7 @@ func (r *Replica) ReadProtectedTimestamps(ctx context.Context) { defer r.maybeUpdateCachedProtectedTS(&ts) r.mu.RLock() defer r.mu.RUnlock() - ts = r.readProtectedTimestampsRLocked(ctx, nil /* f */) + ts = r.readProtectedTimestampsRLocked(ctx) } // ClosedTimestampPolicy returns the closed timestamp policy of the range, which diff --git a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel index 1a46390e209c..0b20be4a049f 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/spanconfig", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/stop", diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache.go b/pkg/kv/kvserver/protectedts/ptcache/cache.go index b379cdf49871..e99d012fdb1b 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -73,6 +74,7 @@ func New(config Config) *Cache { } var _ protectedts.Cache = (*Cache)(nil) +var _ spanconfig.ProtectedTSReader = (*Cache)(nil) // Iterate is part of the protectedts.Cache interface. func (c *Cache) Iterate( @@ -126,6 +128,21 @@ func (c *Cache) Refresh(ctx context.Context, asOf hlc.Timestamp) error { return nil } +// GetProtectionTimestamps is part of the spanconfig.ProtectedTSReader +// interface. +func (c *Cache) GetProtectionTimestamps( + ctx context.Context, sp roachpb.Span, +) (protectionTimestamps []hlc.Timestamp, asOf hlc.Timestamp) { + readAt := c.Iterate(ctx, + sp.Key, + sp.EndKey, + func(rec *ptpb.Record) (wantMore bool) { + protectionTimestamps = append(protectionTimestamps, rec.Timestamp) + return true + }) + return protectionTimestamps, readAt +} + // Start starts the periodic fetching of the Cache. A Cache must not be used // until after it has been started. An error will be returned if it has // already been started. diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index cd251dd372b6..1a5e44862237 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -68,7 +68,7 @@ func TestCacheBasic(t *testing.T) { // Then we'll add a record and make sure it gets seen. sp := tableSpan(42) - r, createdAt := protect(t, tc.Server(0), p, sp) + r, createdAt := protect(t, tc.Server(0), p, s.Clock().Now(), sp) testutils.SucceedsSoon(t, func() error { var coveredBy []*ptpb.Record seenTS := c.Iterate(ctx, sp.Key, sp.EndKey, @@ -140,7 +140,7 @@ func TestRefresh(t *testing.T) { st.verifyCounters(t, 1, 0) // just need to scan meta }) t.Run("needs refresh, with change", func(t *testing.T) { - _, createdAt := protect(t, s, p, metaTableSpan) + _, createdAt := protect(t, s, p, s.Clock().Now(), metaTableSpan) st.resetCounters() require.NoError(t, c.Refresh(ctx, createdAt)) st.verifyCounters(t, 2, 1) // need to scan meta and then scan everything @@ -177,7 +177,7 @@ func TestRefresh(t *testing.T) { require.Regexp(t, "boom", c.Refresh(ctx, s.Clock().Now()).Error()) }) t.Run("error propagates while fetching records", func(t *testing.T) { - protect(t, s, p, metaTableSpan) + protect(t, s, p, s.Clock().Now(), metaTableSpan) st.setFilter(func(ba roachpb.BatchRequest) *roachpb.Error { if scanReq, ok := ba.GetArg(roachpb.Scan); ok { scan := scanReq.(*roachpb.ScanRequest) @@ -192,7 +192,7 @@ func TestRefresh(t *testing.T) { }) t.Run("Iterate does not hold mutex", func(t *testing.T) { inIterate := make(chan chan struct{}) - rec, createdAt := protect(t, s, p, metaTableSpan) + rec, createdAt := protect(t, s, p, s.Clock().Now(), metaTableSpan) require.NoError(t, c.Refresh(ctx, createdAt)) go c.Iterate(ctx, keys.MinKey, keys.MaxKey, func(r *ptpb.Record) (wantMore bool) { if r.ID.GetUUID() != rec.ID.GetUUID() { @@ -271,8 +271,8 @@ func TestQueryRecord(t *testing.T) { waitForAsOfAfter(t, c, hlc.Timestamp{}) // Create two records. sp42 := tableSpan(42) - r1, createdAt1 := protect(t, s, p, sp42) - r2, createdAt2 := protect(t, s, p, sp42) + r1, createdAt1 := protect(t, s, p, s.Clock().Now(), sp42) + r2, createdAt2 := protect(t, s, p, s.Clock().Now(), sp42) // Ensure they both don't exist and that the read timestamps precede the // create timestamps. exists1, asOf := c.QueryRecord(ctx, r1.ID.GetUUID()) @@ -291,7 +291,7 @@ func TestQueryRecord(t *testing.T) { require.True(t, !asOf.Less(createdAt2)) // Release 2 and then create 3. require.NoError(t, p.Release(ctx, nil /* txn */, r2.ID.GetUUID())) - r3, createdAt3 := protect(t, s, p, sp42) + r3, createdAt3 := protect(t, s, p, s.Clock().Now(), sp42) exists2, asOf = c.QueryRecord(ctx, r2.ID.GetUUID()) require.True(t, exists2) require.True(t, asOf.Less(createdAt3)) @@ -329,10 +329,10 @@ func TestIterate(t *testing.T) { sp42 := tableSpan(42) sp43 := tableSpan(43) sp44 := tableSpan(44) - r1, _ := protect(t, s, p, sp42) - r2, _ := protect(t, s, p, sp43) - r3, _ := protect(t, s, p, sp44) - r4, _ := protect(t, s, p, sp42, sp43) + r1, _ := protect(t, s, p, s.Clock().Now(), sp42) + r2, _ := protect(t, s, p, s.Clock().Now(), sp43) + r3, _ := protect(t, s, p, s.Clock().Now(), sp44) + r4, _ := protect(t, s, p, s.Clock().Now(), sp42, sp43) require.NoError(t, c.Refresh(ctx, s.Clock().Now())) t.Run("all", func(t *testing.T) { var recs records @@ -373,6 +373,99 @@ func (recs *records) sorted() []*ptpb.Record { return *recs } +func TestGetProtectionTimestamps(t *testing.T) { + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + // Set the poll interval to be very long. + s := tc.Server(0) + protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) + + ts := func(nanos int) hlc.Timestamp { + return hlc.Timestamp{ + WallTime: int64(nanos), + } + } + sp42 := tableSpan(42) + sp43 := tableSpan(43) + sp44 := tableSpan(44) + sp4243 := roachpb.Span{Key: sp42.Key, EndKey: sp43.EndKey} + + for _, testCase := range []struct { + name string + test func(t *testing.T, p protectedts.Storage, c *ptcache.Cache, cleanup func(...*ptpb.Record)) + }{ + { + name: "multiple records apply to a single span", + test: func(t *testing.T, p protectedts.Storage, c *ptcache.Cache, cleanup func(...*ptpb.Record)) { + r1, _ := protect(t, s, p, ts(10), sp42) + r2, _ := protect(t, s, p, ts(11), sp42) + r3, _ := protect(t, s, p, ts(6), sp42) + require.NoError(t, c.Refresh(ctx, s.Clock().Now())) + + protectionTimestamps, _ := c.GetProtectionTimestamps(ctx, sp42) + sort.Slice(protectionTimestamps, func(i, j int) bool { + return protectionTimestamps[i].Less(protectionTimestamps[j]) + }) + require.Equal(t, []hlc.Timestamp{ts(6), ts(10), ts(11)}, protectionTimestamps) + cleanup(r1, r2, r3) + }, + }, + { + name: "no records apply", + test: func(t *testing.T, p protectedts.Storage, c *ptcache.Cache, cleanup func(...*ptpb.Record)) { + r1, _ := protect(t, s, p, ts(5), sp43) + r2, _ := protect(t, s, p, ts(10), sp44) + require.NoError(t, c.Refresh(ctx, s.Clock().Now())) + protectionTimestamps, _ := c.GetProtectionTimestamps(ctx, sp42) + require.Equal(t, []hlc.Timestamp(nil), protectionTimestamps) + cleanup(r1, r2) + }, + }, + { + name: "multiple overlapping spans multiple records", + test: func(t *testing.T, p protectedts.Storage, c *ptcache.Cache, cleanup func(...*ptpb.Record)) { + r1, _ := protect(t, s, p, ts(10), sp42) + r2, _ := protect(t, s, p, ts(15), sp42) + r3, _ := protect(t, s, p, ts(5), sp43) + r4, _ := protect(t, s, p, ts(6), sp43) + r5, _ := protect(t, s, p, ts(25), keys.EverythingSpan) + // Also add a record that doesn't overlap with the requested span and + // ensure it isn't retrieved below. + r6, _ := protect(t, s, p, ts(20), sp44) + require.NoError(t, c.Refresh(ctx, s.Clock().Now())) + + protectionTimestamps, _ := c.GetProtectionTimestamps(ctx, sp4243) + sort.Slice(protectionTimestamps, func(i, j int) bool { + return protectionTimestamps[i].Less(protectionTimestamps[j]) + }) + require.Equal( + t, []hlc.Timestamp{ts(5), ts(6), ts(10), ts(15), ts(25)}, protectionTimestamps, + ) + cleanup(r1, r2, r3, r4, r5, r6) + }, + }, + } { + t.Run(testCase.name, func(t *testing.T) { + p := ptstorage.WithDatabase(ptstorage.New(s.ClusterSettings(), + s.InternalExecutor().(sqlutil.InternalExecutor), nil /* knobs */), s.DB()) + + c := ptcache.New(ptcache.Config{ + Settings: s.ClusterSettings(), + DB: s.DB(), + Storage: p, + }) + require.NoError(t, c.Start(ctx, tc.Stopper())) + + testCase.test(t, p, c, func(records ...*ptpb.Record) { + for _, r := range records { + require.NoError(t, p.Release(ctx, nil, r.ID.GetUUID())) + } + }) + }) + } +} + func TestSettingChangedLeadsToFetch(t *testing.T) { ctx := context.Background() tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) @@ -425,9 +518,12 @@ func tableSpan(tableID uint32) roachpb.Span { } func protect( - t *testing.T, s serverutils.TestServerInterface, p protectedts.Storage, spans ...roachpb.Span, + t *testing.T, + s serverutils.TestServerInterface, + p protectedts.Storage, + protectTS hlc.Timestamp, + spans ...roachpb.Span, ) (r *ptpb.Record, createdAt hlc.Timestamp) { - protectTS := s.Clock().Now() r = &ptpb.Record{ ID: uuid.MakeV4().GetBytes(), Timestamp: protectTS, diff --git a/pkg/kv/kvserver/protectedts/ptprovider/provider.go b/pkg/kv/kvserver/protectedts/ptprovider/provider.go index c537151bf20e..1f55a2557808 100644 --- a/pkg/kv/kvserver/protectedts/ptprovider/provider.go +++ b/pkg/kv/kvserver/protectedts/ptprovider/provider.go @@ -39,7 +39,8 @@ type Config struct { Knobs *protectedts.TestingKnobs } -type provider struct { +// Provider is the concrete implementation of protectedts.Provider interface. +type Provider struct { protectedts.Storage protectedts.Verifier protectedts.Cache @@ -55,13 +56,15 @@ func New(cfg Config) (protectedts.Provider, error) { storage := ptstorage.New(cfg.Settings, cfg.InternalExecutor, cfg.Knobs) verifier := ptverifier.New(cfg.DB, storage) reconciler := ptreconcile.New(cfg.Settings, cfg.DB, storage, cfg.ReconcileStatusFuncs) - return &provider{ - Storage: storage, - Cache: ptcache.New(ptcache.Config{ - DB: cfg.DB, - Storage: storage, - Settings: cfg.Settings, - }), + cache := ptcache.New(ptcache.Config{ + DB: cfg.DB, + Storage: storage, + Settings: cfg.Settings, + }) + + return &Provider{ + Storage: storage, + Cache: cache, Verifier: verifier, Reconciler: reconciler, Struct: reconciler.Metrics(), @@ -81,13 +84,15 @@ func validateConfig(cfg Config) error { } } -func (p *provider) Start(ctx context.Context, stopper *stop.Stopper) error { +// Start implements the protectedts.Provider interface. +func (p *Provider) Start(ctx context.Context, stopper *stop.Stopper) error { if cache, ok := p.Cache.(*ptcache.Cache); ok { return cache.Start(ctx, stopper) } return nil } -func (p *provider) Metrics() metric.Struct { +// Metrics implements the protectedts.Provider interface. +func (p *Provider) Metrics() metric.Struct { return p.Struct } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 65a593eabf67..bf67cf5926af 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1024,8 +1024,8 @@ func (r *Replica) getImpliedGCThresholdRLocked( // If we have a protected timestamp record which precedes the implied // threshold, use the threshold it implies instead. - if c.earliestRecord != nil && c.earliestRecord.Timestamp.Less(threshold) { - return c.earliestRecord.Timestamp.Prev() + if !c.earliestProtectionTimestamp.IsEmpty() && c.earliestProtectionTimestamp.Less(threshold) { + return c.earliestProtectionTimestamp.Prev() } return threshold } diff --git a/pkg/kv/kvserver/replica_protected_timestamp.go b/pkg/kv/kvserver/replica_protected_timestamp.go index efab180fdc65..48a5eb132424 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp.go +++ b/pkg/kv/kvserver/replica_protected_timestamp.go @@ -15,7 +15,6 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -25,8 +24,8 @@ import ( // cachedProtectedTimestampState is used to cache information about the state // of protected timestamps as they pertain to this replica. The data is // refreshed when the replica examines protected timestamps when being -// considered for gc or when verifying a protected timestamp record. -// It is consulted when determining whether a request can be served. +// considered for gc. It is consulted when determining whether a request can be +// served. type cachedProtectedTimestampState struct { // readAt denotes the timestamp at which this record was read. // It is used to coordinate updates to this field. It is also used to @@ -35,8 +34,8 @@ type cachedProtectedTimestampState struct { // that protected timestamps have not been observed. In this case we must // assume that any protected timestamp could exist to provide the contract // on verify. - readAt hlc.Timestamp - earliestRecord *ptpb.Record + readAt hlc.Timestamp + earliestProtectionTimestamp hlc.Timestamp } // clearIfNotNewer clears the state in ts if it is not newer than the passed @@ -66,31 +65,29 @@ func (r *Replica) maybeUpdateCachedProtectedTS(ts *cachedProtectedTimestampState } func (r *Replica) readProtectedTimestampsRLocked( - ctx context.Context, f func(r *ptpb.Record), + ctx context.Context, ) (ts cachedProtectedTimestampState) { desc := r.descRLocked() gcThreshold := *r.mu.state.GCThreshold - ts.readAt = r.store.protectedtsCache.Iterate(ctx, - roachpb.Key(desc.StartKey), - roachpb.Key(desc.EndKey), - func(rec *ptpb.Record) (wantMore bool) { - // Check if we've already GC'd past the timestamp this record was trying - // to protect, in which case we know that the record does not apply. - // Note that when we implement PROTECT_AT, we'll need to consult some - // replica state here to determine whether the record indeed has been - // applied. - if isValid := gcThreshold.LessEq(rec.Timestamp); !isValid { - return true - } - if f != nil { - f(rec) - } - if ts.earliestRecord == nil || rec.Timestamp.Less(ts.earliestRecord.Timestamp) { - ts.earliestRecord = rec - } - return true - }) + sp := roachpb.Span{ + Key: roachpb.Key(desc.StartKey), + EndKey: roachpb.Key(desc.EndKey), + } + var protectionTimestamps []hlc.Timestamp + protectionTimestamps, ts.readAt = r.store.protectedtsReader.GetProtectionTimestamps(ctx, sp) + earliestTS := hlc.Timestamp{} + for _, protectionTimestamp := range protectionTimestamps { + // Check if the timestamp the record was trying to protect is strictly + // below the GCThreshold, in which case, we know the record does not apply. + if isValid := gcThreshold.LessEq(protectionTimestamp); !isValid { + continue + } + if earliestTS.IsEmpty() || protectionTimestamp.Less(earliestTS) { + earliestTS = protectionTimestamp + } + } + ts.earliestProtectionTimestamp = earliestTS return ts } @@ -126,12 +123,12 @@ func (r *Replica) checkProtectedTimestampsForGC( // read.earliestRecord is the record with the earliest timestamp which is // greater than the existing gcThreshold. - read = r.readProtectedTimestampsRLocked(ctx, nil) + read = r.readProtectedTimestampsRLocked(ctx) gcTimestamp = read.readAt - if read.earliestRecord != nil { + if !read.earliestProtectionTimestamp.IsEmpty() { // NB: we want to allow GC up to the timestamp preceding the earliest valid - // record. - impliedGCTimestamp := gc.TimestampForThreshold(read.earliestRecord.Timestamp.Prev(), gcTTL) + // protection timestamp. + impliedGCTimestamp := gc.TimestampForThreshold(read.earliestProtectionTimestamp.Prev(), gcTTL) if impliedGCTimestamp.Less(gcTimestamp) { gcTimestamp = impliedGCTimestamp } diff --git a/pkg/kv/kvserver/replica_protected_timestamp_test.go b/pkg/kv/kvserver/replica_protected_timestamp_test.go index a70be6270156..d47d29851cf0 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp_test.go +++ b/pkg/kv/kvserver/replica_protected_timestamp_test.go @@ -12,26 +12,23 @@ package kvserver import ( "context" - "fmt" + "math/rand" "testing" "time" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" "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/stop" - "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/stretchr/testify/require" ) // TestCheckProtectedTimestampsForGC exercises -// Replica.checkProtectedTimestampsForGC() at a low level. -// It does so by passing a Replica connected to an already -// shut down store to a variety of test cases. +// Replica.checkProtectedTimestampsForGC() at a low level. It does so by passing +// a Replica connected to an already shut down store to a variety of test cases. func TestCheckProtectedTimestampsForGC(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -44,11 +41,11 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { name string // Note that the store underneath the passed in Replica has been stopped. // This leaves the test to mutate the Replica state as it sees fit. - test func(t *testing.T, r *Replica, mt *manualCache) + test func(t *testing.T, r *Replica, mp *manualPTSReader) }{ { name: "lease is too new", - test: func(t *testing.T, r *Replica, mt *manualCache) { + test: func(t *testing.T, r *Replica, _ *manualPTSReader) { r.mu.state.Lease.Start = r.store.Clock().NowAsClockTimestamp() canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.False(t, canGC) @@ -57,42 +54,30 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { }, { name: "have overlapping but new enough that it's okay", - test: func(t *testing.T, r *Replica, mt *manualCache) { + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { ts := r.store.Clock().Now() - mt.asOf = r.store.Clock().Now().Next() - mt.records = append(mt.records, &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: ts, - DeprecatedSpans: []roachpb.Span{ - { - Key: keys.MinKey, - EndKey: keys.MaxKey, - }, - }, + mp.asOf = r.store.Clock().Now().Next() + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{ts}, }) // We should allow gc to proceed with the normal new threshold if that // threshold is earlier than all of the records. canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) - require.Equal(t, mt.asOf, gcTimestamp) + require.Equal(t, mp.asOf, gcTimestamp) }, }, { // In this case we have a record which protects some data but we can // set the threshold to a later point. name: "have overlapping but can still GC some", - test: func(t *testing.T, r *Replica, mt *manualCache) { + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { ts := r.store.Clock().Now().Add(-11*time.Second.Nanoseconds(), 0) - mt.asOf = r.store.Clock().Now().Next() - mt.records = append(mt.records, &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: ts, - DeprecatedSpans: []roachpb.Span{ - { - Key: keys.MinKey, - EndKey: keys.MaxKey, - }, - }, + mp.asOf = r.store.Clock().Now().Next() + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{ts}, }) // We should allow gc to proceed up to the timestamp which precedes the // protected timestamp. This means we expect a GC timestamp 10 seconds @@ -107,20 +92,14 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { // In this case we have a record which is right up against the GC // threshold. name: "have overlapping but have already GC'd right up to the threshold", - test: func(t *testing.T, r *Replica, mt *manualCache) { + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { r.mu.Lock() th := *r.mu.state.GCThreshold r.mu.Unlock() - mt.asOf = r.store.Clock().Now().Next() - mt.records = append(mt.records, &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: th.Next(), - DeprecatedSpans: []roachpb.Span{ - { - Key: keys.MinKey, - EndKey: keys.MaxKey, - }, - }, + mp.asOf = r.store.Clock().Now().Next() + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{th.Next()}, }) // We should allow GC even if the threshold is already the // predecessor of the earliest valid record. However, the GC @@ -128,85 +107,135 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { // applicable to manually enqueued ranges. canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) + require.Equal(t, newThreshold, oldThreshold) require.True(t, newThreshold.Equal(oldThreshold)) require.Equal(t, th.Add(10*time.Second.Nanoseconds(), 0), gcTimestamp) }, }, { name: "failed record does not prevent GC", - test: func(t *testing.T, r *Replica, mt *manualCache) { + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { ts := r.store.Clock().Now() - id := uuid.MakeV4() thresh := ts.Next() r.mu.state.GCThreshold = &thresh - mt.asOf = thresh.Next() - mt.records = append(mt.records, &ptpb.Record{ - ID: id.GetBytes(), - Timestamp: ts, - DeprecatedSpans: []roachpb.Span{ - { - Key: keys.MinKey, - EndKey: keys.MaxKey, - }, - }, + mp.asOf = thresh.Next() + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{ts}, }) canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) require.True(t, canGC) - require.Equal(t, mt.asOf, gcTimestamp) + require.Equal(t, mp.asOf, gcTimestamp) + }, + }, + { + name: "earliest timestamp is picked when multiple records exist", + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { + ts1 := r.store.Clock().Now().Add(-11*time.Second.Nanoseconds(), 0) + ts2 := r.store.Clock().Now().Add(-20*time.Second.Nanoseconds(), 0) + ts3 := r.store.Clock().Now().Add(-30*time.Second.Nanoseconds(), 0) + mp.asOf = r.store.Clock().Now().Next() + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{ts1, ts2, ts3}, + }) + + // Shuffle the protection timestamps for good measure. + mp.shuffleAllProtectionTimestamps() + // We should allow gc to proceed up to the timestamp which precedes the + // earliest protected timestamp (t3). This means we expect a GC + // timestamp 10 seconds after ts3.Prev() given the policy. + canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) + require.True(t, canGC) + require.False(t, newThreshold.Equal(oldThreshold)) + require.Equal(t, ts3.Prev().Add(10*time.Second.Nanoseconds(), 0), gcTimestamp) + }, + }, + { + // We should be able to move the GC timestamp up if no protection + // timestamps apply. The timestamp moves up till how fresh our reading of + // PTS state is. + name: "no protections apply", + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { + mp.asOf = r.store.Clock().Now().Next() + canGC, _, gcTimestamp, _, _ := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) + require.True(t, canGC) + require.Equal(t, mp.asOf, gcTimestamp) + }, + }, + { + // Set up such that multiple timestamps are present including timestamps + // from failed records (i.e below the GCThreshold). We should be able to + // move the GC timestamp using the earliest protection timestamp that is + // still above the GCThreshold in such a case. + name: "multiple timestamps present including failed", + test: func(t *testing.T, r *Replica, mp *manualPTSReader) { + mp.asOf = r.store.Clock().Now().Next() + thresh := r.mu.state.GCThreshold + ts1 := thresh.Add(-7*time.Second.Nanoseconds(), 0) + ts2 := thresh.Add(-4*time.Second.Nanoseconds(), 0) + ts3 := thresh.Add(14*time.Second.Nanoseconds(), 0) + ts4 := thresh.Add(20*time.Second.Nanoseconds(), 0) + mp.protections = append(mp.protections, manualPTSReaderProtection{ + sp: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + protectionTimestamps: []hlc.Timestamp{ts1, ts2, ts3, ts4}, + }) + mp.shuffleAllProtectionTimestamps() + // We should allow gc to proceed up to the timestamp which precedes the + // earliest protected timestamp (t3) that is still valid. This means we + // expect a GC timestamp 10 seconds after ts3.Prev() given the policy. + canGC, _, gcTimestamp, oldThreshold, newThreshold := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(10)) + require.True(t, canGC) + require.False(t, newThreshold.Equal(oldThreshold)) + require.Equal(t, ts3.Prev().Add(10*time.Second.Nanoseconds(), 0), gcTimestamp) }, }, } { t.Run(testCase.name, func(t *testing.T) { tc := testContext{} tsc := TestStoreConfig(nil) - mc := &manualCache{} - tsc.ProtectedTimestampCache = mc + mp := &manualPTSReader{} + tsc.ProtectedTimestampReader = mp stopper := stop.NewStopper() tc.StartWithStoreConfig(ctx, t, stopper, tsc) stopper.Stop(ctx) - testCase.test(t, tc.repl, mc) + testCase.test(t, tc.repl, mp) }) } } -type manualCache struct { - asOf hlc.Timestamp - records []*ptpb.Record - refresh func(ctx context.Context, asOf hlc.Timestamp) error +type manualPTSReaderProtection struct { + sp roachpb.Span + protectionTimestamps []hlc.Timestamp } -func (c *manualCache) Iterate( - ctx context.Context, start, end roachpb.Key, it protectedts.Iterator, -) hlc.Timestamp { - query := roachpb.Span{Key: start, EndKey: end} - for _, r := range c.records { - for _, sp := range r.DeprecatedSpans { - if query.Overlaps(sp) { - it(r) - break - } - } - } - return c.asOf +type manualPTSReader struct { + asOf hlc.Timestamp + protections []manualPTSReaderProtection } -func (c *manualCache) Refresh(ctx context.Context, asOf hlc.Timestamp) error { - if c.refresh == nil { - c.asOf = asOf - return nil +// GetProtectionTimestamps is part of the spanconfig.ProtectedTSReader +// interface. +func (mp *manualPTSReader) GetProtectionTimestamps( + _ context.Context, sp roachpb.Span, +) (protectionTimestamps []hlc.Timestamp, asOf hlc.Timestamp) { + for _, protection := range mp.protections { + if protection.sp.Overlaps(sp) { + protectionTimestamps = append(protectionTimestamps, protection.protectionTimestamps...) + } } - return c.refresh(ctx, asOf) + return protectionTimestamps, mp.asOf } -func (c *manualCache) QueryRecord( - ctx context.Context, id uuid.UUID, -) (exists bool, asOf hlc.Timestamp) { - for _, r := range c.records { - if r.ID.GetUUID() == id { - return true, c.asOf - } +// shuffleAllProtectionTimestamps shuffles protection timestamps associated with +// all spans. +func (mp *manualPTSReader) shuffleAllProtectionTimestamps() { + for i := range mp.protections { + rand.Shuffle(len(mp.protections[i].protectionTimestamps), func(a, b int) { + mp.protections[i].protectionTimestamps[a], mp.protections[i].protectionTimestamps[b] = + mp.protections[i].protectionTimestamps[b], mp.protections[i].protectionTimestamps[a] + }) } - return false, c.asOf } -var _ protectedts.Cache = (*manualCache)(nil) +var _ spanconfig.ProtectedTSReader = (*manualPTSReader)(nil) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index cb2c980c64e7..8191ecca6952 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -40,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" @@ -246,7 +245,7 @@ func testStoreConfig(clock *hlc.Clock, version roachpb.Version) StoreConfig { CoalescedHeartbeatsInterval: 50 * time.Millisecond, ScanInterval: 10 * time.Minute, HistogramWindowInterval: metric.TestSampleInterval, - ProtectedTimestampCache: protectedts.EmptyCache(clock), + ProtectedTimestampReader: spanconfig.EmptyProtectedTSReader(clock), // Use a constant empty system config, which mirrors the previously // existing logic to install an empty system config in gossip. @@ -732,7 +731,7 @@ type Store struct { limiters batcheval.Limiters txnWaitMetrics *txnwait.Metrics sstSnapshotStorage SSTSnapshotStorage - protectedtsCache protectedts.Cache + protectedtsReader spanconfig.ProtectedTSReader ctSender *sidetransport.Sender // gossipRangeCountdown and leaseRangeCountdown are countdowns of @@ -1044,10 +1043,9 @@ type StoreConfig struct { ExternalStorage cloud.ExternalStorageFactory ExternalStorageFromURI cloud.ExternalStorageFromURIFactory - // ProtectedTimestampCache maintains the state of the protected timestamp - // subsystem. It is queried during the GC process and in the handling of - // AdminVerifyProtectedTimestampRequest. - ProtectedTimestampCache protectedts.Cache + // ProtectedTimestampReader provides a read-only view into the protected + // timestamp subsystem. It is queried during the GC process. + ProtectedTimestampReader spanconfig.ProtectedTSReader // KV Memory Monitor. Must be non-nil for production, and can be nil in some // tests. @@ -1227,7 +1225,7 @@ func NewStore( if err := s.sstSnapshotStorage.Clear(); err != nil { log.Warningf(ctx, "failed to clear snapshot storage: %v", err) } - s.protectedtsCache = cfg.ProtectedTimestampCache + s.protectedtsReader = cfg.ProtectedTimestampReader // On low-CPU instances, a default limit value may still allow ExportRequests // to tie up all cores so cap limiter at cores-1 when setting value is higher. diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 815c2406666c..9f7b394f28ba 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -103,6 +103,7 @@ go_library( "//pkg/kv/kvserver/loqrecovery", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb", "//pkg/kv/kvserver/protectedts", + "//pkg/kv/kvserver/protectedts/ptcache", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/kv/kvserver/protectedts/ptprovider", "//pkg/kv/kvserver/protectedts/ptreconcile", @@ -138,6 +139,7 @@ go_library( "//pkg/spanconfig/spanconfigkvaccessor", "//pkg/spanconfig/spanconfigkvsubscriber", "//pkg/spanconfig/spanconfigmanager", + "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigreconciler", "//pkg/spanconfig/spanconfigsqltranslator", "//pkg/spanconfig/spanconfigsqlwatcher", diff --git a/pkg/server/server.go b/pkg/server/server.go index e2219ffe5a33..ed7629ab468a 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptcache" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptprovider" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptreconcile" serverrangefeed "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rangefeed" @@ -56,6 +57,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" // register jobs declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" @@ -511,36 +513,37 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { systemConfigWatcher := systemconfigwatcher.New( keys.SystemSQLCodec, clock, rangeFeedFactory, &cfg.DefaultZoneConfig, ) + protectedTSReader := spanconfigptsreader.NewAdapter(protectedtsProvider.(*ptprovider.Provider).Cache.(*ptcache.Cache)) storeCfg := kvserver.StoreConfig{ - DefaultSpanConfig: cfg.DefaultZoneConfig.AsSpanConfig(), - Settings: st, - AmbientCtx: cfg.AmbientCtx, - RaftConfig: cfg.RaftConfig, - Clock: clock, - DB: db, - Gossip: g, - NodeLiveness: nodeLiveness, - Transport: raftTransport, - NodeDialer: nodeDialer, - RPCContext: rpcContext, - ScanInterval: cfg.ScanInterval, - ScanMinIdleTime: cfg.ScanMinIdleTime, - ScanMaxIdleTime: cfg.ScanMaxIdleTime, - HistogramWindowInterval: cfg.HistogramWindowInterval(), - StorePool: storePool, - SQLExecutor: internalExecutor, - LogRangeEvents: cfg.EventLogEnabled, - RangeDescriptorCache: distSender.RangeDescriptorCache(), - TimeSeriesDataStore: tsDB, - ClosedTimestampSender: ctSender, - ClosedTimestampReceiver: ctReceiver, - ExternalStorage: externalStorage, - ExternalStorageFromURI: externalStorageFromURI, - ProtectedTimestampCache: protectedtsProvider, - KVMemoryMonitor: kvMemoryMonitor, - RangefeedBudgetFactory: rangeReedBudgetFactory, - SystemConfigProvider: systemConfigWatcher, + DefaultSpanConfig: cfg.DefaultZoneConfig.AsSpanConfig(), + Settings: st, + AmbientCtx: cfg.AmbientCtx, + RaftConfig: cfg.RaftConfig, + Clock: clock, + DB: db, + Gossip: g, + NodeLiveness: nodeLiveness, + Transport: raftTransport, + NodeDialer: nodeDialer, + RPCContext: rpcContext, + ScanInterval: cfg.ScanInterval, + ScanMinIdleTime: cfg.ScanMinIdleTime, + ScanMaxIdleTime: cfg.ScanMaxIdleTime, + HistogramWindowInterval: cfg.HistogramWindowInterval(), + StorePool: storePool, + SQLExecutor: internalExecutor, + LogRangeEvents: cfg.EventLogEnabled, + RangeDescriptorCache: distSender.RangeDescriptorCache(), + TimeSeriesDataStore: tsDB, + ClosedTimestampSender: ctSender, + ClosedTimestampReceiver: ctReceiver, + ExternalStorage: externalStorage, + ExternalStorageFromURI: externalStorageFromURI, + ProtectedTimestampReader: protectedTSReader, + KVMemoryMonitor: kvMemoryMonitor, + RangefeedBudgetFactory: rangeReedBudgetFactory, + SystemConfigProvider: systemConfigWatcher, } var spanConfig struct { diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 16df0ebdd5ca..4d74301ace69 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -397,3 +397,31 @@ func (u Update) Deletion() bool { func (u Update) Addition() bool { return !u.Deletion() } + +// ProtectedTSReader is the read-only portion for querying protected +// timestamp information. It doubles up as an adaptor interface for +// protectedts.Cache. +type ProtectedTSReader interface { + // GetProtectionTimestamps returns all protected timestamps that apply to any + // part of the given key span. The time at which this protected timestamp + // state is valid is returned as well. + GetProtectionTimestamps(ctx context.Context, sp roachpb.Span) ( + protectionTimestamps []hlc.Timestamp, asOf hlc.Timestamp, + ) +} + +// EmptyProtectedTSReader returns a ProtectedTSReader which contains no records +// and is always up-to date. This is intended for testing. +func EmptyProtectedTSReader(c *hlc.Clock) ProtectedTSReader { + return (*emptyProtectedTSReader)(c) +} + +type emptyProtectedTSReader hlc.Clock + +// GetProtectionTimestamps is part of the spanconfig.ProtectedTSReader +// interface. +func (r *emptyProtectedTSReader) GetProtectionTimestamps( + _ context.Context, _ roachpb.Span, +) ([]hlc.Timestamp, hlc.Timestamp) { + return nil, (*hlc.Clock)(r).Now() +} diff --git a/pkg/spanconfig/spanconfigptsreader/BUILD.bazel b/pkg/spanconfig/spanconfigptsreader/BUILD.bazel new file mode 100644 index 000000000000..5f3af5de4f52 --- /dev/null +++ b/pkg/spanconfig/spanconfigptsreader/BUILD.bazel @@ -0,0 +1,15 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "spanconfigptsreader", + srcs = ["adapter.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/protectedts/ptcache", + "//pkg/roachpb", + "//pkg/spanconfig", + "//pkg/util/hlc", + "@com_github_cockroachdb_errors//:errors", + ], +) diff --git a/pkg/spanconfig/spanconfigptsreader/adapter.go b/pkg/spanconfig/spanconfigptsreader/adapter.go new file mode 100644 index 000000000000..7ebf4f66a1f1 --- /dev/null +++ b/pkg/spanconfig/spanconfigptsreader/adapter.go @@ -0,0 +1,78 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package spanconfigptsreader + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptcache" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +// adapter implements the spanconfig.ProtectedTSReader interface and is intended +// as a bridge between the old and new protected timestamp subsystems in KV. +// +// V1 of the protected timestamp subsystem only allowed protections to be set +// over spans belonging to the system tenant. These protections were cached by +// each node in the cluster by ptcache.Cache. V2 of the subsystem allows +// protections to be set on all spans (including secondary tenant spans) and are +// cached in the spanconfig.Store maintained by the spanconfig.KVSubscriber. In +// release 22.1 both the old and new subsystem co-exist, and as such, +// protections that apply to system tenant spans may be present in either the +// ptcache.Cache or spanconfig.KVSubscriber. This adapter struct encapsulates +// protected timestamp information from both these sources behind a single +// interface. +// +// TODO(arul): In 22.2, we would have completely migrated away from the old +// subsystem, and we'd be able to get rid of this interface. +// +// TODO(arul): Add the KVSubscriber here as well and actually encapsulate PTS +// information from both these sources as described above; This will happen once +// we make the KVSubscriber implement the spanconfig.ProtectedTSReader +// interface. +type adapter struct { + cache *ptcache.Cache +} + +var _ spanconfig.ProtectedTSReader = &adapter{} + +// NewAdapter returns an adapter that implements spanconfig.ProtectedTSReader. +func NewAdapter(cache *ptcache.Cache) spanconfig.ProtectedTSReader { + return &adapter{ + cache: cache, + } +} + +// GetProtectionTimestamps is part of the spanconfig.ProtectedTSReader +// interface. +func (a *adapter) GetProtectionTimestamps( + ctx context.Context, sp roachpb.Span, +) (protectionTimestamps []hlc.Timestamp, asOf hlc.Timestamp) { + return a.cache.GetProtectionTimestamps(ctx, sp) +} + +// TestingRefreshPTSState refreshes the in-memory protected timestamp state to +// at least asOf. +// TODO(arul): Once we wrap the KVSubscriber in this adapter interface, we'll +// need to ensure that the subscriber is at-least as caught up as the supplied +// asOf timestamp as well. +func TestingRefreshPTSState( + ctx context.Context, protectedTSReader spanconfig.ProtectedTSReader, asOf hlc.Timestamp, +) error { + a, ok := protectedTSReader.(*adapter) + if !ok { + return errors.AssertionFailedf("could not convert protectedts.Provider to ptprovider.Provider") + } + return a.cache.Refresh(ctx, asOf) +} From 5eba198fa623f07a8215a1177722d626f41d9bac Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Tue, 1 Mar 2022 17:04:24 -0500 Subject: [PATCH 4/6] Unary Complement execution has different results when the parameters are different fixes https://github.com/cockroachdb/cockroach/issues/74493 Release note (sql change): Return ambiguous unary operator error for ambiguous input like ~'1' which can be interpreted as an integer (resulting in -2) or a bit string (resulting in 0). Release justification: Improves a confusing error message saying that an operator is invalid instead of ambiguous. --- .../logictest/testdata/logic_test/operator | 44 +++++++++++++++++++ pkg/sql/sem/tree/overload.go | 11 +++++ pkg/sql/sem/tree/overload_test.go | 4 +- pkg/sql/sem/tree/type_check_test.go | 2 +- 4 files changed, 58 insertions(+), 3 deletions(-) diff --git a/pkg/sql/logictest/testdata/logic_test/operator b/pkg/sql/logictest/testdata/logic_test/operator index c55255e9c31e..8766db95715a 100644 --- a/pkg/sql/logictest/testdata/logic_test/operator +++ b/pkg/sql/logictest/testdata/logic_test/operator @@ -3,3 +3,47 @@ SELECT |/ -1.0::float query error cannot take square root of a negative number SELECT |/ -1.0::decimal + +query I +SELECT ~-1; +---- +0 + +query I +SELECT ~0; +---- +-1 + +query I +SELECT ~1; +---- +-2 + +query I +SELECT ~2; +---- +-3 + +query T +SELECT ~B'0'; +---- +1 + +query T +SELECT ~B'1'; +---- +0 + +statement error lexical error +SELECT ~B'2'; + +statement error ambiguous unary operator +SELECT ~'0'; + +statement error ambiguous unary operator +SELECT ~'1'; + +query I +SELECT ~2; +---- +-3 diff --git a/pkg/sql/sem/tree/overload.go b/pkg/sql/sem/tree/overload.go index 5415911a0c41..7a5a1ec741a4 100644 --- a/pkg/sql/sem/tree/overload.go +++ b/pkg/sql/sem/tree/overload.go @@ -750,6 +750,10 @@ func typeCheckOverloadedExprs( // The fourth heuristic is to prefer candidates that accepts the "best" // mutual type in the resolvable type set of all constants. if bestConstType, ok := commonConstantType(s.exprs, s.constIdxs); ok { + // In case all overloads are filtered out at this step, + // keep track of previous overload indexes to return ambiguous error (>1 overloads) + // instead of unsupported error (0 overloads) when applicable. + prevOverloadIdxs := s.overloadIdxs for _, i := range s.constIdxs { s.overloadIdxs = filterOverloads(s.overloads, s.overloadIdxs, func(o overloadImpl) bool { @@ -757,6 +761,13 @@ func typeCheckOverloadedExprs( }) } if ok, typedExprs, fns, err := checkReturn(ctx, semaCtx, &s); ok { + if len(fns) == 0 { + var overloadImpls []overloadImpl + for i := range prevOverloadIdxs { + overloadImpls = append(overloadImpls, s.overloads[i]) + } + return typedExprs, overloadImpls, err + } return typedExprs, fns, err } if homogeneousTyp != nil { diff --git a/pkg/sql/sem/tree/overload_test.go b/pkg/sql/sem/tree/overload_test.go index 0525efdd8081..5402e64c9a74 100644 --- a/pkg/sql/sem/tree/overload_test.go +++ b/pkg/sql/sem/tree/overload_test.go @@ -187,7 +187,7 @@ func TestTypeCheckOverloadedExprs(t *testing.T) { {nil, []Expr{intConst("1")}, []overloadImpl{unaryIntFn, unaryIntFn}, ambiguous, false}, {nil, []Expr{intConst("1")}, []overloadImpl{unaryIntFn, unaryFloatFn}, unaryIntFn, false}, {nil, []Expr{decConst("1.0")}, []overloadImpl{unaryIntFn, unaryDecimalFn}, unaryDecimalFn, false}, - {nil, []Expr{decConst("1.0")}, []overloadImpl{unaryIntFn, unaryFloatFn}, unsupported, false}, + {nil, []Expr{decConst("1.0")}, []overloadImpl{unaryIntFn, unaryFloatFn}, ambiguous, false}, {nil, []Expr{intConst("1")}, []overloadImpl{unaryIntFn, binaryIntFn}, unaryIntFn, false}, {nil, []Expr{intConst("1")}, []overloadImpl{unaryFloatFn, unaryStringFn}, unaryFloatFn, false}, {nil, []Expr{intConst("1")}, []overloadImpl{unaryStringFn, binaryIntFn}, unsupported, false}, @@ -249,7 +249,7 @@ func TestTypeCheckOverloadedExprs(t *testing.T) { {nil, []Expr{NewDInt(1), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, binaryIntFn, false}, {nil, []Expr{NewDFloat(1), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, unsupported, false}, {nil, []Expr{intConst("1"), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, binaryIntFn, false}, - {nil, []Expr{decConst("1.0"), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, unsupported, false}, // Limitation. + {nil, []Expr{decConst("1.0"), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, ambiguous, false}, // Limitation. {types.Date, []Expr{NewDInt(1), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, binaryIntDateFn, false}, {types.Date, []Expr{NewDFloat(1), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, unsupported, false}, {types.Date, []Expr{intConst("1"), placeholder(1)}, []overloadImpl{binaryIntFn, binaryIntDateFn}, binaryIntDateFn, false}, diff --git a/pkg/sql/sem/tree/type_check_test.go b/pkg/sql/sem/tree/type_check_test.go index e30c0b7abceb..14e7e30b2389 100644 --- a/pkg/sql/sem/tree/type_check_test.go +++ b/pkg/sql/sem/tree/type_check_test.go @@ -244,7 +244,7 @@ func TestTypeCheckError(t *testing.T) { expr string expected string }{ - {`'1' + '2'`, `unsupported binary operator:`}, + {`'1' + '2'`, `ambiguous binary operator:`}, {`'a' + 0`, `unsupported binary operator:`}, {`1.1 # 3.1`, `unsupported binary operator:`}, {`~0.1`, `unsupported unary operator:`}, From 89091fea10ee032bd635a10e981561448f02a219 Mon Sep 17 00:00:00 2001 From: Adam Storm Date: Wed, 23 Feb 2022 09:53:53 -0500 Subject: [PATCH 5/6] settings: Add syntax for cluster settings Before this commit, there was no syntax to SET or SHOW cluster settings which exist for a given tenant. This commit adds the following syntax: * ALTER TENANT SET CLUSTER SETTING = * ALTER TENANT ALL SET CLUSTER SETTING = * ALTER TENANT RESET CLUSTER SETTING * ALTER TENANT ALL RESET CLUSTER SETTING * SHOW CLUSTER SETTING FOR TENANT * SHOW [ALL] CLUSTER SETTINGS FOR TENANT Note that the syntax is added but the underlying commands are currently unimplemented. The implementation of these commands will come with a subsequent commit. Release note (sql change): Added syntax for modifying cluster settings at the tenant level. Release justification: Completion of feature for 22.1. --- docs/generated/sql/bnf/BUILD.bazel | 1 + docs/generated/sql/bnf/alter_stmt.bnf | 1 + .../sql/bnf/alter_tenant_csetting_stmt.bnf | 5 + .../sql/bnf/show_cluster_setting.bnf | 4 + docs/generated/sql/bnf/stmt_block.bnf | 49 ++-- pkg/gen/docs.bzl | 1 + pkg/sql/BUILD.bazel | 1 + pkg/sql/alter_tenant.go | 90 +++++++ pkg/sql/delegate/show_all_cluster_settings.go | 5 + .../testdata/logic_test/cluster_settings | 53 ++++ pkg/sql/opaque.go | 3 + pkg/sql/parser/help_test.go | 9 + pkg/sql/parser/sql.y | 92 ++++++- pkg/sql/parser/testdata/alter_tenant | 31 +++ pkg/sql/parser/testdata/show | 48 ++++ pkg/sql/randgen/mutator.go | 2 +- pkg/sql/sem/tree/BUILD.bazel | 1 + pkg/sql/sem/tree/alter_tenant.go | 53 ++++ pkg/sql/sem/tree/set.go | 9 +- pkg/sql/sem/tree/show.go | 15 +- pkg/sql/sem/tree/stmt.go | 10 + pkg/sql/sem/tree/walk.go | 20 ++ pkg/sql/set_cluster_setting.go | 28 +- pkg/sql/show_cluster_setting.go | 5 + pkg/sql/walk.go | 244 +++++++++--------- 25 files changed, 622 insertions(+), 158 deletions(-) create mode 100644 docs/generated/sql/bnf/alter_tenant_csetting_stmt.bnf create mode 100644 pkg/sql/alter_tenant.go create mode 100644 pkg/sql/parser/testdata/alter_tenant create mode 100644 pkg/sql/sem/tree/alter_tenant.go diff --git a/docs/generated/sql/bnf/BUILD.bazel b/docs/generated/sql/bnf/BUILD.bazel index 706e8a799fc0..f4594adb359f 100644 --- a/docs/generated/sql/bnf/BUILD.bazel +++ b/docs/generated/sql/bnf/BUILD.bazel @@ -38,6 +38,7 @@ FILES = [ "alter_table_partition_by", "alter_table_set_schema_stmt", "alter_table_stmt", + "alter_tenant_csetting_stmt", "alter_type", "alter_view", "alter_view_owner_stmt", diff --git a/docs/generated/sql/bnf/alter_stmt.bnf b/docs/generated/sql/bnf/alter_stmt.bnf index 50297fc69343..c41f5092df99 100644 --- a/docs/generated/sql/bnf/alter_stmt.bnf +++ b/docs/generated/sql/bnf/alter_stmt.bnf @@ -1,3 +1,4 @@ alter_stmt ::= alter_ddl_stmt | alter_role_stmt + | alter_tenant_csetting_stmt diff --git a/docs/generated/sql/bnf/alter_tenant_csetting_stmt.bnf b/docs/generated/sql/bnf/alter_tenant_csetting_stmt.bnf new file mode 100644 index 000000000000..e605b3613be1 --- /dev/null +++ b/docs/generated/sql/bnf/alter_tenant_csetting_stmt.bnf @@ -0,0 +1,5 @@ +alter_tenant_csetting_stmt ::= + 'ALTER' 'TENANT' iconst64 'SET' 'CLUSTER' 'SETTING' var_name to_or_eq var_value + | 'ALTER' 'TENANT' 'ALL' 'SET' 'CLUSTER' 'SETTING' var_name to_or_eq var_value + | 'ALTER' 'TENANT' iconst64 'RESET' 'CLUSTER' 'SETTING' var_name + | 'ALTER' 'TENANT' 'ALL' 'RESET' 'CLUSTER' 'SETTING' var_name diff --git a/docs/generated/sql/bnf/show_cluster_setting.bnf b/docs/generated/sql/bnf/show_cluster_setting.bnf index 1faef0311955..5ff43300647c 100644 --- a/docs/generated/sql/bnf/show_cluster_setting.bnf +++ b/docs/generated/sql/bnf/show_cluster_setting.bnf @@ -4,3 +4,7 @@ show_csettings_stmt ::= | 'SHOW' 'ALL' 'CLUSTER' 'SETTINGS' | 'SHOW' 'CLUSTER' 'SETTINGS' | 'SHOW' 'PUBLIC' 'CLUSTER' 'SETTINGS' + | 'SHOW' 'CLUSTER' 'SETTING' var_name 'FOR' 'TENANT' iconst64 + | 'SHOW' 'ALL' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 + | 'SHOW' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 + | 'SHOW' 'PUBLIC' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 2141dc8aa5ea..f44d865a9a25 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -135,6 +135,7 @@ fetch_cursor_stmt ::= alter_stmt ::= alter_ddl_stmt | alter_role_stmt + | alter_tenant_csetting_stmt backup_stmt ::= 'BACKUP' opt_backup_targets 'INTO' sconst_or_placeholder 'IN' string_or_placeholder_opt_list opt_as_of_clause opt_with_backup_options @@ -456,6 +457,12 @@ alter_role_stmt ::= | 'ALTER' 'ROLE_ALL' 'ALL' opt_in_database set_or_reset_clause | 'ALTER' 'USER_ALL' 'ALL' opt_in_database set_or_reset_clause +alter_tenant_csetting_stmt ::= + 'ALTER' 'TENANT' iconst64 'SET' 'CLUSTER' 'SETTING' var_name to_or_eq var_value + | 'ALTER' 'TENANT' 'ALL' 'SET' 'CLUSTER' 'SETTING' var_name to_or_eq var_value + | 'ALTER' 'TENANT' iconst64 'RESET' 'CLUSTER' 'SETTING' var_name + | 'ALTER' 'TENANT' 'ALL' 'RESET' 'CLUSTER' 'SETTING' var_name + opt_backup_targets ::= targets @@ -715,6 +722,10 @@ show_csettings_stmt ::= | 'SHOW' 'ALL' 'CLUSTER' 'SETTINGS' | 'SHOW' 'CLUSTER' 'SETTINGS' | 'SHOW' 'PUBLIC' 'CLUSTER' 'SETTINGS' + | 'SHOW' 'CLUSTER' 'SETTING' var_name 'FOR' 'TENANT' iconst64 + | 'SHOW' 'ALL' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 + | 'SHOW' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 + | 'SHOW' 'PUBLIC' 'CLUSTER' 'SETTINGS' 'FOR' 'TENANT' iconst64 show_databases_stmt ::= 'SHOW' 'DATABASES' with_comment @@ -1501,6 +1512,18 @@ set_or_reset_clause ::= | 'RESET_ALL' 'ALL' | 'RESET' session_var +var_name ::= + name + | name attrs + +to_or_eq ::= + '=' + | 'TO' + +var_value ::= + a_expr + | extra_var_value + as_of_clause ::= 'AS' 'OF' 'SYSTEM' 'TIME' a_expr @@ -1671,10 +1694,6 @@ session_var ::= | 'LC_CTYPE' | 'TIME' 'ZONE' -var_name ::= - name - | name attrs - restore_options_list ::= ( restore_options ) ( ( ',' restore_options ) )* @@ -1716,14 +1735,6 @@ set_rest_more ::= set_rest ::= generic_set -to_or_eq ::= - '=' - | 'TO' - -var_value ::= - a_expr - | extra_var_value - with_comment ::= 'WITH' 'COMMENT' | @@ -1991,6 +2002,13 @@ alter_backup_cmds ::= role_options ::= ( role_option ) ( ( role_option ) )* +attrs ::= + ( '.' unrestricted_name ) ( ( '.' unrestricted_name ) )* + +extra_var_value ::= + 'ON' + | cockroachdb_extra_reserved_keyword + backup_options ::= 'ENCRYPTION_PASSPHRASE' '=' string_or_placeholder | 'REVISION_HISTORY' @@ -2225,9 +2243,6 @@ column_name ::= session_var_parts ::= ( '.' 'identifier' ) ( ( '.' 'identifier' ) )* -attrs ::= - ( '.' unrestricted_name ) ( ( '.' unrestricted_name ) )* - restore_options ::= 'ENCRYPTION_PASSPHRASE' '=' string_or_placeholder | 'KMS' '=' string_or_placeholder_opt_list @@ -2272,10 +2287,6 @@ offset_clause ::= generic_set ::= var_name to_or_eq var_list -extra_var_value ::= - 'ON' - | cockroachdb_extra_reserved_keyword - targets_roles ::= 'ROLE' role_spec_list | 'SCHEMA' schema_name_list diff --git a/pkg/gen/docs.bzl b/pkg/gen/docs.bzl index 7af9c489b395..7a9d5b858340 100644 --- a/pkg/gen/docs.bzl +++ b/pkg/gen/docs.bzl @@ -50,6 +50,7 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:alter_table_partition_by.bnf", "//docs/generated/sql/bnf:alter_table_set_schema_stmt.bnf", "//docs/generated/sql/bnf:alter_table_stmt.bnf", + "//docs/generated/sql/bnf:alter_tenant_csetting_stmt.bnf", "//docs/generated/sql/bnf:alter_type.bnf", "//docs/generated/sql/bnf:alter_view.bnf", "//docs/generated/sql/bnf:alter_view_owner_stmt.bnf", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 03d19ae024b5..76d8272c53be 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "alter_table_locality.go", "alter_table_owner.go", "alter_table_set_schema.go", + "alter_tenant.go", "alter_type.go", "analyze_expr.go", "apply_join.go", diff --git a/pkg/sql/alter_tenant.go b/pkg/sql/alter_tenant.go new file mode 100644 index 000000000000..57d0f85fa7a5 --- /dev/null +++ b/pkg/sql/alter_tenant.go @@ -0,0 +1,90 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + "strings" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" + "github.com/cockroachdb/errors" +) + +// alterTenantSetClusterSettingNode represents an +// ALTER TENANT ... SET CLUSTER SETTING statement. +type alterTenantSetClusterSettingNode struct { + name string + tenantID roachpb.TenantID + tenantAll bool + st *cluster.Settings + setting settings.NonMaskedSetting + // If value is nil, the setting should be reset. + value tree.TypedExpr +} + +// AlterTenantSetClusterSetting sets tenant level session variables. +// Privileges: super user. +func (p *planner) AlterTenantSetClusterSetting( + ctx context.Context, n *tree.AlterTenantSetClusterSetting, +) (planNode, error) { + name := strings.ToLower(n.Name) + st := p.EvalContext().Settings + v, ok := settings.Lookup(name, settings.LookupForLocalAccess, p.ExecCfg().Codec.ForSystemTenant()) + if !ok { + return nil, errors.Errorf("unknown cluster setting '%s'", name) + } + + if err := checkPrivilegesForSetting(ctx, p, name, "set"); err != nil { + return nil, err + } + + setting, ok := v.(settings.NonMaskedSetting) + if !ok { + return nil, errors.AssertionFailedf("expected writable setting, got %T", v) + } + + // Error out if we're trying to call this from a non-system tenant or if + // we're trying to set a system-only variable. + if !p.execCfg.Codec.ForSystemTenant() { + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, + "ALTER TENANT can only be called by system operators") + } + if setting.Class() == settings.SystemOnly { + return nil, pgerror.Newf(pgcode.InsufficientPrivilege, + "%s is a system-only setting and must be set at the host cluster level using SET CLUSTER SETTING", name) + } + + value, err := p.getAndValidateTypedClusterSetting(ctx, name, n.Value, setting) + if err != nil { + return nil, err + } + + node := alterTenantSetClusterSettingNode{ + name: name, tenantID: n.TenantID, tenantAll: n.TenantAll, st: st, + setting: setting, value: value, + } + return &node, nil +} + +func (n *alterTenantSetClusterSettingNode) startExec(params runParams) error { + return unimplemented.NewWithIssue(73857, + `unimplemented: tenant-level cluster settings not supported`) +} + +func (n *alterTenantSetClusterSettingNode) Next(_ runParams) (bool, error) { return false, nil } +func (n *alterTenantSetClusterSettingNode) Values() tree.Datums { return nil } +func (n *alterTenantSetClusterSettingNode) Close(_ context.Context) {} diff --git a/pkg/sql/delegate/show_all_cluster_settings.go b/pkg/sql/delegate/show_all_cluster_settings.go index 4f31c36105fb..d1115dcd15ba 100644 --- a/pkg/sql/delegate/show_all_cluster_settings.go +++ b/pkg/sql/delegate/show_all_cluster_settings.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" ) func (d *delegator) delegateShowClusterSettingList( @@ -24,6 +25,10 @@ func (d *delegator) delegateShowClusterSettingList( if err != nil { return nil, err } + if stmt.TenantID.IsSet() { + return nil, unimplemented.NewWithIssue(73857, + `unimplemented: tenant-level cluster settings not supported`) + } hasModify, err := d.catalog.HasRoleOption(d.ctx, roleoption.MODIFYCLUSTERSETTING) if err != nil { return nil, err diff --git a/pkg/sql/logictest/testdata/logic_test/cluster_settings b/pkg/sql/logictest/testdata/logic_test/cluster_settings index 4ebdabe7ba28..fe1a731a9e81 100644 --- a/pkg/sql/logictest/testdata/logic_test/cluster_settings +++ b/pkg/sql/logictest/testdata/logic_test/cluster_settings @@ -22,6 +22,9 @@ SET CLUSTER SETTING sql.conn.max_read_buffer_message_size = '1b' statement ok SET CLUSTER SETTING sql.conn.max_read_buffer_message_size = '64MB' +statement ok +RESET CLUSTER SETTING sql.conn.max_read_buffer_message_size + # Test permissions for modifying cluster settings. user testuser @@ -162,3 +165,53 @@ SET CLUSTER SETTING kv.snapshot_rebalance.max_rate = '10Mib' onlyif config 3node-tenant statement error unknown setting SHOW CLUSTER SETTING kv.snapshot_rebalance.max_rate + +# Test a cluster setting which is SystemOnly +subtest tenant-cluster-settings + +# Skip this test when inside a tenant, as this setting won't be visible in +# there. +skipif config 3node-tenant +statement error admission.kv.enabled is a system-only setting and must be set at the host cluster level using SET CLUSTER SETTING +ALTER TENANT 10 SET CLUSTER SETTING admission.kv.enabled=true + +onlyif config 3node-tenant +statement error ALTER TENANT can only be called by system operators +ALTER TENANT 10 SET CLUSTER SETTING server.mem_profile.total_dump_size_limit='10M' + +skipif config 3node-tenant +statement error unimplemented +ALTER TENANT 10 SET CLUSTER SETTING server.mem_profile.total_dump_size_limit='10M' + +skipif config 3node-tenant +statement error unimplemented +ALTER TENANT ALL SET CLUSTER SETTING server.mem_profile.total_dump_size_limit='10M' + +skipif config 3node-tenant +statement error unimplemented +ALTER TENANT 10 RESET CLUSTER SETTING server.mem_profile.total_dump_size_limit + +skipif config 3node-tenant +statement error unimplemented +ALTER TENANT ALL RESET CLUSTER SETTING server.mem_profile.total_dump_size_limit + +skipif config 3node-tenant +statement error pq: at or near "EOF": syntax error: invalid tenant ID +ALTER TENANT 0 SET CLUSTER SETTING server.mem_profile.total_dump_size_limit='10M' + +skipif config 3node-tenant +statement error pq: at or near "EOF": syntax error: invalid tenant ID +ALTER TENANT 0 RESET CLUSTER SETTING server.mem_profile.total_dump_size_limit + +statement error unimplemented +SHOW CLUSTER SETTING server.mem_profile.total_dump_size_limit FOR TENANT 10 + +statement error unimplemented +SHOW CLUSTER SETTINGS FOR TENANT 10 + +statement error unimplemented +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 10 + +statement error unimplemented +SHOW ALL CLUSTER SETTINGS FOR TENANT 10 + diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index e170fabfecb9..50f24c967a99 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -105,6 +105,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.AlterTableOwner(ctx, n) case *tree.AlterTableSetSchema: return p.AlterTableSetSchema(ctx, n) + case *tree.AlterTenantSetClusterSetting: + return p.AlterTenantSetClusterSetting(ctx, n) case *tree.AlterType: return p.AlterType(ctx, n) case *tree.AlterRole: @@ -251,6 +253,7 @@ func init() { &tree.AlterTableLocality{}, &tree.AlterTableOwner{}, &tree.AlterTableSetSchema{}, + &tree.AlterTenantSetClusterSetting{}, &tree.AlterType{}, &tree.AlterSequence{}, &tree.AlterRole{}, diff --git a/pkg/sql/parser/help_test.go b/pkg/sql/parser/help_test.go index e0ae7109e9ee..3d8e60c1a2ce 100644 --- a/pkg/sql/parser/help_test.go +++ b/pkg/sql/parser/help_test.go @@ -50,6 +50,11 @@ func TestContextualHelp(t *testing.T) { {`ALTER TABLE blah RENAME TO blih ??`, `ALTER TABLE`}, {`ALTER TABLE blah SPLIT AT (SELECT 1) ??`, `ALTER TABLE`}, + {`ALTER TENANT 1 SET CLUSTER SETTING ???`, `ALTER TENANT`}, + {`ALTER TENANT 1 RESET CLUSTER SETTING ???`, `ALTER TENANT`}, + {`ALTER TENANT ALL SET CLUSTER SETTING ???`, `ALTER TENANT`}, + {`ALTER TENANT ALL RESET CLUSTER SETTING ???`, `ALTER TENANT`}, + {`ALTER TYPE ??`, `ALTER TYPE`}, {`ALTER TYPE t ??`, `ALTER TYPE`}, {`ALTER TYPE t ADD VALUE ??`, `ALTER TYPE`}, @@ -327,6 +332,10 @@ func TestContextualHelp(t *testing.T) { {`SHOW CLUSTER SETTING all ??`, `SHOW CLUSTER SETTING`}, {`SHOW ALL CLUSTER ??`, `SHOW CLUSTER SETTING`}, + {`SHOW CLUSTER SETTING a FOR TENANT ??`, `SHOW CLUSTER SETTING`}, + {`SHOW ALL CLUSTER SETTINGS FOR TENANT ??`, `SHOW CLUSTER SETTING`}, + {`SHOW CLUSTER SETTINGS FOR TENANT ??`, `SHOW CLUSTER SETTING`}, + {`SHOW PUBLIC CLUSTER SETTINGS FOR TENANT ??`, `SHOW CLUSTER SETTING`}, {`SHOW COLUMNS FROM ??`, `SHOW COLUMNS`}, {`SHOW COLUMNS FROM foo ??`, `SHOW COLUMNS`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 831629b874fe..9a3433b213c4 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -958,6 +958,9 @@ func (u *sqlSymUnion) fetchCursor() *tree.FetchCursor { %type alter_table_locality_stmt %type alter_table_owner_stmt +// ALTER TENANT CLUSTER SETTINGS +%type alter_tenant_csetting_stmt + // ALTER PARTITION %type alter_zone_partition_stmt @@ -1572,10 +1575,11 @@ stmt: // %Help: ALTER // %Category: Group -// %Text: ALTER TABLE, ALTER INDEX, ALTER VIEW, ALTER SEQUENCE, ALTER DATABASE, ALTER USER, ALTER ROLE, ALTER DEFAULT PRIVILEGES +// %Text: ALTER TABLE, ALTER INDEX, ALTER VIEW, ALTER SEQUENCE, ALTER DATABASE, ALTER USER, ALTER ROLE, ALTER DEFAULT PRIVILEGES, ALTER TENANT alter_stmt: alter_ddl_stmt // help texts in sub-rule | alter_role_stmt // EXTEND WITH HELP: ALTER ROLE +| alter_tenant_csetting_stmt // EXTEND WITH HELP: ALTER TENANT | alter_unsupported_stmt | ALTER error // SHOW HELP: ALTER @@ -4947,6 +4951,56 @@ set_csetting_stmt: } | SET CLUSTER error // SHOW HELP: SET CLUSTER SETTING +// %Help: ALTER TENANT - alter tenant configuration +// %Category: Cfg +// %Text: +// ALTER TENANT { | ALL } SET CLUSTER SETTING { TO | = } +// ALTER TENANT { | ALL } RESET CLUSTER SETTING +// %SeeAlso: SET CLUSTER SETTING +alter_tenant_csetting_stmt: + ALTER TENANT iconst64 SET CLUSTER SETTING var_name to_or_eq var_value + { + tenID := uint64($3.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.AlterTenantSetClusterSetting{ + Name: strings.Join($7.strs(), "."), + Value: $9.expr(), + TenantID: roachpb.MakeTenantID(tenID), + } + } +| ALTER TENANT ALL SET CLUSTER SETTING var_name to_or_eq var_value + { + $$.val = &tree.AlterTenantSetClusterSetting{ + Name: strings.Join($7.strs(), "."), + Value: $9.expr(), + TenantAll: true, + } + } +| ALTER TENANT iconst64 RESET CLUSTER SETTING var_name + { + tenID := uint64($3.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.AlterTenantSetClusterSetting{ + Name: strings.Join($7.strs(), "."), + Value: tree.DefaultVal{}, + TenantID: roachpb.MakeTenantID(tenID), + } + } +| ALTER TENANT ALL RESET CLUSTER SETTING var_name + { + $$.val = &tree.AlterTenantSetClusterSetting{ + Name: strings.Join($7.strs(), "."), + Value: tree.DefaultVal{}, + TenantAll: true, + } + } +| ALTER TENANT error // SHOW HELP: ALTER TENANT + + to_or_eq: '=' | TO @@ -5642,8 +5696,8 @@ show_backup_stmt: // %Help: SHOW CLUSTER SETTING - display cluster settings // %Category: Cfg // %Text: -// SHOW CLUSTER SETTING -// SHOW [ PUBLIC | ALL ] CLUSTER SETTINGS +// SHOW CLUSTER SETTING [ FOR TENANT ] +// SHOW [ PUBLIC | ALL ] CLUSTER SETTINGS [ FOR TENANT ] // %SeeAlso: WEBDOCS/cluster-settings.html show_csettings_stmt: SHOW CLUSTER SETTING var_name @@ -5668,6 +5722,38 @@ show_csettings_stmt: { $$.val = &tree.ShowClusterSettingList{} } +| SHOW CLUSTER SETTING var_name FOR TENANT iconst64 + { + tenID := uint64($7.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.ShowClusterSetting{Name: strings.Join($4.strs(), "."), TenantID: roachpb.MakeTenantID(tenID)} + } +| SHOW ALL CLUSTER SETTINGS FOR TENANT iconst64 + { + tenID := uint64($7.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.ShowClusterSettingList{All: true, TenantID: roachpb.MakeTenantID(tenID)} + } +| SHOW CLUSTER SETTINGS FOR TENANT iconst64 + { + tenID := uint64($6.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.ShowClusterSettingList{TenantID: roachpb.MakeTenantID(tenID)} + } +| SHOW PUBLIC CLUSTER SETTINGS FOR TENANT iconst64 + { + tenID := uint64($7.int64()) + if tenID == 0 { + return setErr(sqllex, errors.New("invalid tenant ID")) + } + $$.val = &tree.ShowClusterSettingList{TenantID: roachpb.MakeTenantID(tenID)} + } | SHOW PUBLIC CLUSTER error // SHOW HELP: SHOW CLUSTER SETTING // %Help: SHOW COLUMNS - list columns in relation diff --git a/pkg/sql/parser/testdata/alter_tenant b/pkg/sql/parser/testdata/alter_tenant new file mode 100644 index 000000000000..c1a2881ca452 --- /dev/null +++ b/pkg/sql/parser/testdata/alter_tenant @@ -0,0 +1,31 @@ +parse +ALTER TENANT 1 SET CLUSTER SETTING a = 3 +---- +ALTER TENANT 1 SET CLUSTER SETTING a = 3 +ALTER TENANT 1 SET CLUSTER SETTING a = (3) -- fully parenthesized +ALTER TENANT 1 SET CLUSTER SETTING a = _ -- literals removed +ALTER TENANT 1 SET CLUSTER SETTING a = 3 -- identifiers removed + +parse +ALTER TENANT ALL SET CLUSTER SETTING a = 3 +---- +ALTER TENANT ALL SET CLUSTER SETTING a = 3 +ALTER TENANT ALL SET CLUSTER SETTING a = (3) -- fully parenthesized +ALTER TENANT ALL SET CLUSTER SETTING a = _ -- literals removed +ALTER TENANT ALL SET CLUSTER SETTING a = 3 -- identifiers removed + +parse +ALTER TENANT 1 RESET CLUSTER SETTING a +---- +ALTER TENANT 1 SET CLUSTER SETTING a = DEFAULT -- normalized! +ALTER TENANT 1 SET CLUSTER SETTING a = (DEFAULT) -- fully parenthesized +ALTER TENANT 1 SET CLUSTER SETTING a = DEFAULT -- literals removed +ALTER TENANT 1 SET CLUSTER SETTING a = DEFAULT -- identifiers removed + +parse +ALTER TENANT ALL RESET CLUSTER SETTING a +---- +ALTER TENANT ALL SET CLUSTER SETTING a = DEFAULT -- normalized! +ALTER TENANT ALL SET CLUSTER SETTING a = (DEFAULT) -- fully parenthesized +ALTER TENANT ALL SET CLUSTER SETTING a = DEFAULT -- literals removed +ALTER TENANT ALL SET CLUSTER SETTING a = DEFAULT -- identifiers removed diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index 44d84d143057..a5b88da84ae8 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -126,6 +126,54 @@ SHOW PUBLIC CLUSTER SETTINGS -- fully parenthesized SHOW PUBLIC CLUSTER SETTINGS -- literals removed SHOW PUBLIC CLUSTER SETTINGS -- identifiers removed +parse +SHOW CLUSTER SETTING a FOR TENANT 1 +---- +SHOW CLUSTER SETTING a FOR TENANT 1 +SHOW CLUSTER SETTING a FOR TENANT 1 -- fully parenthesized +SHOW CLUSTER SETTING a FOR TENANT 1 -- literals removed +SHOW CLUSTER SETTING a FOR TENANT 1 -- identifiers removed + +parse +EXPLAIN SHOW CLUSTER SETTING a FOR TENANT 1 +---- +EXPLAIN SHOW CLUSTER SETTING a FOR TENANT 1 +EXPLAIN SHOW CLUSTER SETTING a FOR TENANT 1 -- fully parenthesized +EXPLAIN SHOW CLUSTER SETTING a FOR TENANT 1 -- literals removed +EXPLAIN SHOW CLUSTER SETTING a FOR TENANT 1 -- identifiers removed + +parse +SHOW ALL CLUSTER SETTINGS FOR TENANT 1 +---- +SHOW ALL CLUSTER SETTINGS FOR TENANT 1 +SHOW ALL CLUSTER SETTINGS FOR TENANT 1 -- fully parenthesized +SHOW ALL CLUSTER SETTINGS FOR TENANT 1 -- literals removed +SHOW ALL CLUSTER SETTINGS FOR TENANT 1 -- identifiers removed + +parse +SHOW CLUSTER SETTING a FOR TENANT 1 +---- +SHOW CLUSTER SETTING a FOR TENANT 1 +SHOW CLUSTER SETTING a FOR TENANT 1 -- fully parenthesized +SHOW CLUSTER SETTING a FOR TENANT 1 -- literals removed +SHOW CLUSTER SETTING a FOR TENANT 1 -- identifiers removed + +parse +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 +---- +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- fully parenthesized +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- literals removed +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- identifiers removed + +parse +SHOW CLUSTER SETTINGS FOR TENANT 1 +---- +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- normalized! +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- fully parenthesized +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- literals removed +SHOW PUBLIC CLUSTER SETTINGS FOR TENANT 1 -- identifiers removed + parse SHOW DATABASES ---- diff --git a/pkg/sql/randgen/mutator.go b/pkg/sql/randgen/mutator.go index eac35e1b6437..2f3d69244b83 100644 --- a/pkg/sql/randgen/mutator.go +++ b/pkg/sql/randgen/mutator.go @@ -641,7 +641,7 @@ func postgresMutator(rng *rand.Rand, q string) string { var postgresStatementMutator MultiStatementMutation = func(rng *rand.Rand, stmts []tree.Statement) (mutated []tree.Statement, changed bool) { for _, stmt := range stmts { switch stmt := stmt.(type) { - case *tree.SetClusterSetting, *tree.SetVar: + case *tree.SetClusterSetting, *tree.SetVar, *tree.AlterTenantSetClusterSetting: continue case *tree.CreateTable: if stmt.PartitionByTable != nil { diff --git a/pkg/sql/sem/tree/BUILD.bazel b/pkg/sql/sem/tree/BUILD.bazel index c907535a623b..00b326e27b1d 100644 --- a/pkg/sql/sem/tree/BUILD.bazel +++ b/pkg/sql/sem/tree/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "alter_schema.go", "alter_sequence.go", "alter_table.go", + "alter_tenant.go", "alter_type.go", "analyze.go", "annotation.go", diff --git a/pkg/sql/sem/tree/alter_tenant.go b/pkg/sql/sem/tree/alter_tenant.go new file mode 100644 index 000000000000..1ec4485d32bb --- /dev/null +++ b/pkg/sql/sem/tree/alter_tenant.go @@ -0,0 +1,53 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package tree + +import ( + "fmt" + + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// AlterTenantSetClusterSetting represents an ALTER TENANT +// SET CLUSTER SETTING statement. +type AlterTenantSetClusterSetting struct { + Name string + Value Expr + TenantID roachpb.TenantID + TenantAll bool +} + +// Format implements the NodeFormatter interface. +func (n *AlterTenantSetClusterSetting) Format(ctx *FmtCtx) { + if n.TenantAll { + ctx.WriteString("ALTER TENANT ALL ") + } else if n.TenantID.IsSet() { + ctx.WriteString(fmt.Sprintf("ALTER TENANT %d ", n.TenantID.ToUint64())) + } + ctx.WriteString("SET CLUSTER SETTING ") + + // Cluster setting names never contain PII and should be distinguished + // for feature tracking purposes. + ctx.WithFlags(ctx.flags & ^FmtAnonymize & ^FmtMarkRedactionNode, func() { + ctx.FormatNameP(&n.Name) + }) + + ctx.WriteString(" = ") + + switch v := n.Value.(type) { + case *DBool, *DInt: + ctx.WithFlags(ctx.flags & ^FmtAnonymize & ^FmtMarkRedactionNode, func() { + ctx.FormatNode(v) + }) + default: + ctx.FormatNode(v) + } +} diff --git a/pkg/sql/sem/tree/set.go b/pkg/sql/sem/tree/set.go index a6827fb87fc4..b045ba2b5c62 100644 --- a/pkg/sql/sem/tree/set.go +++ b/pkg/sql/sem/tree/set.go @@ -19,6 +19,8 @@ package tree +import "github.com/cockroachdb/cockroach/pkg/roachpb" + // SetVar represents a SET or RESET statement. type SetVar struct { Name string @@ -65,13 +67,16 @@ func (node *SetVar) Format(ctx *FmtCtx) { // SetClusterSetting represents a SET CLUSTER SETTING statement. type SetClusterSetting struct { - Name string - Value Expr + Name string + Value Expr + TenantID roachpb.TenantID + TenantAll bool } // Format implements the NodeFormatter interface. func (node *SetClusterSetting) Format(ctx *FmtCtx) { ctx.WriteString("SET CLUSTER SETTING ") + // Cluster setting names never contain PII and should be distinguished // for feature tracking purposes. ctx.WithFlags(ctx.flags & ^FmtAnonymize & ^FmtMarkRedactionNode, func() { diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 8e3615e3f0e1..1832d799df30 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -22,6 +22,7 @@ package tree import ( "fmt" + "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" ) @@ -42,7 +43,8 @@ func (node *ShowVar) Format(ctx *FmtCtx) { // ShowClusterSetting represents a SHOW CLUSTER SETTING statement. type ShowClusterSetting struct { - Name string + Name string + TenantID roachpb.TenantID } // Format implements the NodeFormatter interface. @@ -53,12 +55,17 @@ func (node *ShowClusterSetting) Format(ctx *FmtCtx) { ctx.WithFlags(ctx.flags & ^FmtAnonymize & ^FmtMarkRedactionNode, func() { ctx.FormatNameP(&node.Name) }) + if node.TenantID.IsSet() { + s := fmt.Sprintf(" FOR TENANT %d", node.TenantID.ToUint64()) + ctx.WriteString(s) + } } // ShowClusterSettingList represents a SHOW [ALL|PUBLIC] CLUSTER SETTINGS statement. type ShowClusterSettingList struct { // All indicates whether to include non-public settings in the output. - All bool + All bool + TenantID roachpb.TenantID } // Format implements the NodeFormatter interface. @@ -70,6 +77,10 @@ func (node *ShowClusterSettingList) Format(ctx *FmtCtx) { } ctx.WriteString(qual) ctx.WriteString(" CLUSTER SETTINGS") + if node.TenantID.IsSet() { + s := fmt.Sprintf(" FOR TENANT %d", node.TenantID.ToUint64()) + ctx.WriteString(s) + } } // BackupDetails represents the type of details to display for a SHOW BACKUP diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index f5735107091d..350823752745 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -342,6 +342,15 @@ func (*AlterSchema) StatementTag() string { return "ALTER SCHEMA" } func (*AlterSchema) hiddenFromShowQueries() {} +// StatementReturnType implements the Statement interface. +func (*AlterTenantSetClusterSetting) StatementReturnType() StatementReturnType { return Ack } + +// StatementType implements the Statement interface. +func (*AlterTenantSetClusterSetting) StatementType() StatementType { return TypeDCL } + +// StatementTag returns a short string identifying the type of statement. +func (*AlterTenantSetClusterSetting) StatementTag() string { return "ALTER TENANT SET CLUSTER SETTING" } + // StatementReturnType implements the Statement interface. func (*AlterType) StatementReturnType() StatementReturnType { return DDL } @@ -1751,6 +1760,7 @@ func (n *AlterTableSetVisible) String() string { return AsString(n) } func (n *AlterTableSetNotNull) String() string { return AsString(n) } func (n *AlterTableOwner) String() string { return AsString(n) } func (n *AlterTableSetSchema) String() string { return AsString(n) } +func (n *AlterTenantSetClusterSetting) String() string { return AsString(n) } func (n *AlterType) String() string { return AsString(n) } func (n *AlterRole) String() string { return AsString(n) } func (n *AlterRoleSet) String() string { return AsString(n) } diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index c748532f5202..53e1e154b5be 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -868,6 +868,25 @@ func walkReturningClause(v Visitor, clause ReturningClause) (ReturningClause, bo } } +// copyNode makes a copy of this Statement without recursing in any child Statements. +func (n *AlterTenantSetClusterSetting) copyNode() *AlterTenantSetClusterSetting { + stmtCopy := *n + return &stmtCopy +} + +// walkStmt is part of the walkableStmt interface. +func (n *AlterTenantSetClusterSetting) walkStmt(v Visitor) Statement { + ret := n + if n.Value != nil { + e, changed := WalkExpr(v, n.Value) + if changed { + ret = n.copyNode() + ret.Value = e + } + } + return ret +} + // copyNode makes a copy of this Statement without recursing in any child Statements. func (stmt *Backup) copyNode() *Backup { stmtCopy := *stmt @@ -1571,6 +1590,7 @@ func (stmt *BeginTransaction) walkStmt(v Visitor) Statement { return ret } +var _ walkableStmt = &AlterTenantSetClusterSetting{} var _ walkableStmt = &CreateTable{} var _ walkableStmt = &Backup{} var _ walkableStmt = &Delete{} diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index 19a9d8512163..688d73e31efc 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -121,11 +121,25 @@ func (p *planner) SetClusterSetting( return nil, errors.Errorf("cluster setting '%s' is currently overridden by the operator", name) } + value, err := p.getAndValidateTypedClusterSetting(ctx, name, n.Value, setting) + if err != nil { + return nil, err + } + + csNode := setClusterSettingNode{ + name: name, st: st, setting: setting, value: value, + versionUpgradeHook: p.execCfg.VersionUpgradeHook, + } + return &csNode, nil +} + +func (p *planner) getAndValidateTypedClusterSetting( + ctx context.Context, name string, expr tree.Expr, setting settings.NonMaskedSetting, +) (tree.TypedExpr, error) { var value tree.TypedExpr - if n.Value != nil { + if expr != nil { // For DEFAULT, let the value reference be nil. That's a RESET in disguise. - if _, ok := n.Value.(tree.DefaultVal); !ok { - expr := n.Value + if _, ok := expr.(tree.DefaultVal); !ok { expr = paramparse.UnresolvedNameToStrVal(expr) var requiredType *types.T @@ -172,18 +186,12 @@ func (p *planner) SetClusterSetting( } return nil, err } - value = typed } else if _, isVersionSetting := setting.(*settings.VersionSetting); isVersionSetting { return nil, errors.New("cannot RESET cluster version setting") } } - - csNode := setClusterSettingNode{ - name: name, st: st, setting: setting, value: value, - versionUpgradeHook: p.execCfg.VersionUpgradeHook, - } - return &csNode, nil + return value, nil } func (n *setClusterSettingNode) startExec(params runParams) error { diff --git a/pkg/sql/show_cluster_setting.go b/pkg/sql/show_cluster_setting.go index 511cb122054c..ea8555e36e63 100644 --- a/pkg/sql/show_cluster_setting.go +++ b/pkg/sql/show_cluster_setting.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/duration" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -117,6 +118,10 @@ func (p *planner) showVersionSetting( func (p *planner) ShowClusterSetting( ctx context.Context, n *tree.ShowClusterSetting, ) (planNode, error) { + if n.TenantID.IsSet() { + return nil, unimplemented.NewWithIssue(73857, + `unimplemented: tenant-level cluster settings not supported`) + } name := strings.ToLower(n.Name) st := p.ExecCfg().Settings val, ok := settings.Lookup( diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 6beefefdef35..2a545edfd792 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -215,6 +215,7 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { n.sourcePlan = v.visit(n.sourcePlan) } + case *alterTenantSetClusterSettingNode: case *createViewNode: case *setVarNode: case *setClusterSettingNode: @@ -333,125 +334,126 @@ func nodeName(plan planNode) string { // strings are constant and not precomputed so that the type names can // be changed without changing the output of "EXPLAIN". var planNodeNames = map[reflect.Type]string{ - reflect.TypeOf(&alterDatabaseOwnerNode{}): "alter database owner", - reflect.TypeOf(&alterDatabaseAddRegionNode{}): "alter database add region", - reflect.TypeOf(&alterDatabasePrimaryRegionNode{}): "alter database primary region", - reflect.TypeOf(&alterDatabasePlacementNode{}): "alter database placement", - reflect.TypeOf(&alterDatabaseSurvivalGoalNode{}): "alter database survive", - reflect.TypeOf(&alterDatabaseDropRegionNode{}): "alter database drop region", - reflect.TypeOf(&alterDefaultPrivilegesNode{}): "alter default privileges", - reflect.TypeOf(&alterIndexNode{}): "alter index", - reflect.TypeOf(&alterSequenceNode{}): "alter sequence", - reflect.TypeOf(&alterSchemaNode{}): "alter schema", - reflect.TypeOf(&alterTableNode{}): "alter table", - reflect.TypeOf(&alterTableOwnerNode{}): "alter table owner", - reflect.TypeOf(&alterTableSetLocalityNode{}): "alter table set locality", - reflect.TypeOf(&alterTableSetSchemaNode{}): "alter table set schema", - reflect.TypeOf(&alterTypeNode{}): "alter type", - reflect.TypeOf(&alterRoleNode{}): "alter role", - reflect.TypeOf(&alterRoleSetNode{}): "alter role set var", - reflect.TypeOf(&applyJoinNode{}): "apply join", - reflect.TypeOf(&bufferNode{}): "buffer", - reflect.TypeOf(&cancelQueriesNode{}): "cancel queries", - reflect.TypeOf(&cancelSessionsNode{}): "cancel sessions", - reflect.TypeOf(&changePrivilegesNode{}): "change privileges", - reflect.TypeOf(&commentOnColumnNode{}): "comment on column", - reflect.TypeOf(&commentOnConstraintNode{}): "comment on constraint", - reflect.TypeOf(&commentOnDatabaseNode{}): "comment on database", - reflect.TypeOf(&commentOnIndexNode{}): "comment on index", - reflect.TypeOf(&commentOnTableNode{}): "comment on table", - reflect.TypeOf(&commentOnSchemaNode{}): "comment on schema", - reflect.TypeOf(&controlJobsNode{}): "control jobs", - reflect.TypeOf(&controlSchedulesNode{}): "control schedules", - reflect.TypeOf(&createDatabaseNode{}): "create database", - reflect.TypeOf(&createExtensionNode{}): "create extension", - reflect.TypeOf(&createIndexNode{}): "create index", - reflect.TypeOf(&createSequenceNode{}): "create sequence", - reflect.TypeOf(&createSchemaNode{}): "create schema", - reflect.TypeOf(&createStatsNode{}): "create statistics", - reflect.TypeOf(&createTableNode{}): "create table", - reflect.TypeOf(&createTypeNode{}): "create type", - reflect.TypeOf(&CreateRoleNode{}): "create user/role", - reflect.TypeOf(&createViewNode{}): "create view", - reflect.TypeOf(&delayedNode{}): "virtual table", - reflect.TypeOf(&deleteNode{}): "delete", - reflect.TypeOf(&deleteRangeNode{}): "delete range", - reflect.TypeOf(&distinctNode{}): "distinct", - reflect.TypeOf(&dropDatabaseNode{}): "drop database", - reflect.TypeOf(&dropIndexNode{}): "drop index", - reflect.TypeOf(&dropSequenceNode{}): "drop sequence", - reflect.TypeOf(&dropSchemaNode{}): "drop schema", - reflect.TypeOf(&dropTableNode{}): "drop table", - reflect.TypeOf(&dropTypeNode{}): "drop type", - reflect.TypeOf(&DropRoleNode{}): "drop user/role", - reflect.TypeOf(&dropViewNode{}): "drop view", - reflect.TypeOf(&errorIfRowsNode{}): "error if rows", - reflect.TypeOf(&explainPlanNode{}): "explain plan", - reflect.TypeOf(&explainVecNode{}): "explain vectorized", - reflect.TypeOf(&explainDDLNode{}): "explain ddl", - reflect.TypeOf(&exportNode{}): "export", - reflect.TypeOf(&fetchNode{}): "fetch", - reflect.TypeOf(&filterNode{}): "filter", - reflect.TypeOf(&GrantRoleNode{}): "grant role", - reflect.TypeOf(&groupNode{}): "group", - reflect.TypeOf(&hookFnNode{}): "plugin", - reflect.TypeOf(&indexJoinNode{}): "index join", - reflect.TypeOf(&insertNode{}): "insert", - reflect.TypeOf(&insertFastPathNode{}): "insert fast path", - reflect.TypeOf(&invertedFilterNode{}): "inverted filter", - reflect.TypeOf(&invertedJoinNode{}): "inverted join", - reflect.TypeOf(&joinNode{}): "join", - reflect.TypeOf(&limitNode{}): "limit", - reflect.TypeOf(&lookupJoinNode{}): "lookup join", - reflect.TypeOf(&max1RowNode{}): "max1row", - reflect.TypeOf(&ordinalityNode{}): "ordinality", - reflect.TypeOf(&projectSetNode{}): "project set", - reflect.TypeOf(&reassignOwnedByNode{}): "reassign owned by", - reflect.TypeOf(&dropOwnedByNode{}): "drop owned by", - reflect.TypeOf(&recursiveCTENode{}): "recursive cte", - reflect.TypeOf(&refreshMaterializedViewNode{}): "refresh materialized view", - reflect.TypeOf(&relocateNode{}): "relocate", - reflect.TypeOf(&relocateRange{}): "relocate range", - reflect.TypeOf(&renameColumnNode{}): "rename column", - reflect.TypeOf(&renameDatabaseNode{}): "rename database", - reflect.TypeOf(&renameIndexNode{}): "rename index", - reflect.TypeOf(&renameTableNode{}): "rename table", - reflect.TypeOf(&reparentDatabaseNode{}): "reparent database", - reflect.TypeOf(&renderNode{}): "render", - reflect.TypeOf(&resetAllNode{}): "reset all", - reflect.TypeOf(&RevokeRoleNode{}): "revoke role", - reflect.TypeOf(&rowCountNode{}): "count", - reflect.TypeOf(&rowSourceToPlanNode{}): "row source to plan node", - reflect.TypeOf(&saveTableNode{}): "save table", - reflect.TypeOf(&scanBufferNode{}): "scan buffer", - reflect.TypeOf(&scanNode{}): "scan", - reflect.TypeOf(&scatterNode{}): "scatter", - reflect.TypeOf(&scrubNode{}): "scrub", - reflect.TypeOf(&sequenceSelectNode{}): "sequence select", - reflect.TypeOf(&serializeNode{}): "run", - reflect.TypeOf(&setClusterSettingNode{}): "set cluster setting", - reflect.TypeOf(&setVarNode{}): "set", - reflect.TypeOf(&setZoneConfigNode{}): "configure zone", - reflect.TypeOf(&showFingerprintsNode{}): "show fingerprints", - reflect.TypeOf(&showTraceNode{}): "show trace for", - reflect.TypeOf(&showTraceReplicaNode{}): "replica trace", - reflect.TypeOf(&showVarNode{}): "show", - reflect.TypeOf(&sortNode{}): "sort", - reflect.TypeOf(&splitNode{}): "split", - reflect.TypeOf(&topKNode{}): "top-k", - reflect.TypeOf(&unsplitNode{}): "unsplit", - reflect.TypeOf(&unsplitAllNode{}): "unsplit all", - reflect.TypeOf(&spoolNode{}): "spool", - reflect.TypeOf(&truncateNode{}): "truncate", - reflect.TypeOf(&unaryNode{}): "emptyrow", - reflect.TypeOf(&unionNode{}): "union", - reflect.TypeOf(&updateNode{}): "update", - reflect.TypeOf(&upsertNode{}): "upsert", - reflect.TypeOf(&valuesNode{}): "values", - reflect.TypeOf(&virtualTableNode{}): "virtual table values", - reflect.TypeOf(&vTableLookupJoinNode{}): "virtual table lookup join", - reflect.TypeOf(&windowNode{}): "window", - reflect.TypeOf(&zeroNode{}): "norows", - reflect.TypeOf(&zigzagJoinNode{}): "zigzag join", - reflect.TypeOf(&schemaChangePlanNode{}): "schema change", + reflect.TypeOf(&alterDatabaseOwnerNode{}): "alter database owner", + reflect.TypeOf(&alterDatabaseAddRegionNode{}): "alter database add region", + reflect.TypeOf(&alterDatabasePrimaryRegionNode{}): "alter database primary region", + reflect.TypeOf(&alterDatabasePlacementNode{}): "alter database placement", + reflect.TypeOf(&alterDatabaseSurvivalGoalNode{}): "alter database survive", + reflect.TypeOf(&alterDatabaseDropRegionNode{}): "alter database drop region", + reflect.TypeOf(&alterDefaultPrivilegesNode{}): "alter default privileges", + reflect.TypeOf(&alterIndexNode{}): "alter index", + reflect.TypeOf(&alterSequenceNode{}): "alter sequence", + reflect.TypeOf(&alterSchemaNode{}): "alter schema", + reflect.TypeOf(&alterTableNode{}): "alter table", + reflect.TypeOf(&alterTableOwnerNode{}): "alter table owner", + reflect.TypeOf(&alterTableSetLocalityNode{}): "alter table set locality", + reflect.TypeOf(&alterTableSetSchemaNode{}): "alter table set schema", + reflect.TypeOf(&alterTenantSetClusterSettingNode{}): "alter tenant set cluster setting", + reflect.TypeOf(&alterTypeNode{}): "alter type", + reflect.TypeOf(&alterRoleNode{}): "alter role", + reflect.TypeOf(&alterRoleSetNode{}): "alter role set var", + reflect.TypeOf(&applyJoinNode{}): "apply join", + reflect.TypeOf(&bufferNode{}): "buffer", + reflect.TypeOf(&cancelQueriesNode{}): "cancel queries", + reflect.TypeOf(&cancelSessionsNode{}): "cancel sessions", + reflect.TypeOf(&changePrivilegesNode{}): "change privileges", + reflect.TypeOf(&commentOnColumnNode{}): "comment on column", + reflect.TypeOf(&commentOnConstraintNode{}): "comment on constraint", + reflect.TypeOf(&commentOnDatabaseNode{}): "comment on database", + reflect.TypeOf(&commentOnIndexNode{}): "comment on index", + reflect.TypeOf(&commentOnTableNode{}): "comment on table", + reflect.TypeOf(&commentOnSchemaNode{}): "comment on schema", + reflect.TypeOf(&controlJobsNode{}): "control jobs", + reflect.TypeOf(&controlSchedulesNode{}): "control schedules", + reflect.TypeOf(&createDatabaseNode{}): "create database", + reflect.TypeOf(&createExtensionNode{}): "create extension", + reflect.TypeOf(&createIndexNode{}): "create index", + reflect.TypeOf(&createSequenceNode{}): "create sequence", + reflect.TypeOf(&createSchemaNode{}): "create schema", + reflect.TypeOf(&createStatsNode{}): "create statistics", + reflect.TypeOf(&createTableNode{}): "create table", + reflect.TypeOf(&createTypeNode{}): "create type", + reflect.TypeOf(&CreateRoleNode{}): "create user/role", + reflect.TypeOf(&createViewNode{}): "create view", + reflect.TypeOf(&delayedNode{}): "virtual table", + reflect.TypeOf(&deleteNode{}): "delete", + reflect.TypeOf(&deleteRangeNode{}): "delete range", + reflect.TypeOf(&distinctNode{}): "distinct", + reflect.TypeOf(&dropDatabaseNode{}): "drop database", + reflect.TypeOf(&dropIndexNode{}): "drop index", + reflect.TypeOf(&dropSequenceNode{}): "drop sequence", + reflect.TypeOf(&dropSchemaNode{}): "drop schema", + reflect.TypeOf(&dropTableNode{}): "drop table", + reflect.TypeOf(&dropTypeNode{}): "drop type", + reflect.TypeOf(&DropRoleNode{}): "drop user/role", + reflect.TypeOf(&dropViewNode{}): "drop view", + reflect.TypeOf(&errorIfRowsNode{}): "error if rows", + reflect.TypeOf(&explainPlanNode{}): "explain plan", + reflect.TypeOf(&explainVecNode{}): "explain vectorized", + reflect.TypeOf(&explainDDLNode{}): "explain ddl", + reflect.TypeOf(&exportNode{}): "export", + reflect.TypeOf(&fetchNode{}): "fetch", + reflect.TypeOf(&filterNode{}): "filter", + reflect.TypeOf(&GrantRoleNode{}): "grant role", + reflect.TypeOf(&groupNode{}): "group", + reflect.TypeOf(&hookFnNode{}): "plugin", + reflect.TypeOf(&indexJoinNode{}): "index join", + reflect.TypeOf(&insertNode{}): "insert", + reflect.TypeOf(&insertFastPathNode{}): "insert fast path", + reflect.TypeOf(&invertedFilterNode{}): "inverted filter", + reflect.TypeOf(&invertedJoinNode{}): "inverted join", + reflect.TypeOf(&joinNode{}): "join", + reflect.TypeOf(&limitNode{}): "limit", + reflect.TypeOf(&lookupJoinNode{}): "lookup join", + reflect.TypeOf(&max1RowNode{}): "max1row", + reflect.TypeOf(&ordinalityNode{}): "ordinality", + reflect.TypeOf(&projectSetNode{}): "project set", + reflect.TypeOf(&reassignOwnedByNode{}): "reassign owned by", + reflect.TypeOf(&dropOwnedByNode{}): "drop owned by", + reflect.TypeOf(&recursiveCTENode{}): "recursive cte", + reflect.TypeOf(&refreshMaterializedViewNode{}): "refresh materialized view", + reflect.TypeOf(&relocateNode{}): "relocate", + reflect.TypeOf(&relocateRange{}): "relocate range", + reflect.TypeOf(&renameColumnNode{}): "rename column", + reflect.TypeOf(&renameDatabaseNode{}): "rename database", + reflect.TypeOf(&renameIndexNode{}): "rename index", + reflect.TypeOf(&renameTableNode{}): "rename table", + reflect.TypeOf(&reparentDatabaseNode{}): "reparent database", + reflect.TypeOf(&renderNode{}): "render", + reflect.TypeOf(&resetAllNode{}): "reset all", + reflect.TypeOf(&RevokeRoleNode{}): "revoke role", + reflect.TypeOf(&rowCountNode{}): "count", + reflect.TypeOf(&rowSourceToPlanNode{}): "row source to plan node", + reflect.TypeOf(&saveTableNode{}): "save table", + reflect.TypeOf(&scanBufferNode{}): "scan buffer", + reflect.TypeOf(&scanNode{}): "scan", + reflect.TypeOf(&scatterNode{}): "scatter", + reflect.TypeOf(&scrubNode{}): "scrub", + reflect.TypeOf(&sequenceSelectNode{}): "sequence select", + reflect.TypeOf(&serializeNode{}): "run", + reflect.TypeOf(&setClusterSettingNode{}): "set cluster setting", + reflect.TypeOf(&setVarNode{}): "set", + reflect.TypeOf(&setZoneConfigNode{}): "configure zone", + reflect.TypeOf(&showFingerprintsNode{}): "show fingerprints", + reflect.TypeOf(&showTraceNode{}): "show trace for", + reflect.TypeOf(&showTraceReplicaNode{}): "replica trace", + reflect.TypeOf(&showVarNode{}): "show", + reflect.TypeOf(&sortNode{}): "sort", + reflect.TypeOf(&splitNode{}): "split", + reflect.TypeOf(&topKNode{}): "top-k", + reflect.TypeOf(&unsplitNode{}): "unsplit", + reflect.TypeOf(&unsplitAllNode{}): "unsplit all", + reflect.TypeOf(&spoolNode{}): "spool", + reflect.TypeOf(&truncateNode{}): "truncate", + reflect.TypeOf(&unaryNode{}): "emptyrow", + reflect.TypeOf(&unionNode{}): "union", + reflect.TypeOf(&updateNode{}): "update", + reflect.TypeOf(&upsertNode{}): "upsert", + reflect.TypeOf(&valuesNode{}): "values", + reflect.TypeOf(&virtualTableNode{}): "virtual table values", + reflect.TypeOf(&vTableLookupJoinNode{}): "virtual table lookup join", + reflect.TypeOf(&windowNode{}): "window", + reflect.TypeOf(&zeroNode{}): "norows", + reflect.TypeOf(&zigzagJoinNode{}): "zigzag join", + reflect.TypeOf(&schemaChangePlanNode{}): "schema change", } From 3ba6dcb6b23953e859d8d87d3014ad968d82e137 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Tue, 1 Mar 2022 16:05:38 -0500 Subject: [PATCH 6/6] kvserver: fix race in durability callback queueing in raftLogTruncator The existing code admitted the following interleaving between thread-1, running the async raft log truncation, and thread-2 which is running a new durabilityAdvancedCallback. thread-1: executes queued := t.mu.queuedDurabilityCB and sees queued is false thread-2: sees t.mu.runningTruncation is true and sets t.mu.queuedDurabilityCB = true thread-1: Sets t.mu.runningTruncation = false and returns Now the queued callback will never run. This can happen in tests that wait for truncation before doing the next truncation step, because they will stop waiting once the truncation is observed on a Replica, which happens before any of the steps listed above for thread-1. Fixes #77046 Release justification: Bug fix Release note: None --- pkg/kv/kvserver/raft_log_truncator.go | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/pkg/kv/kvserver/raft_log_truncator.go b/pkg/kv/kvserver/raft_log_truncator.go index ad0aef0c485d..47c34d7e612d 100644 --- a/pkg/kv/kvserver/raft_log_truncator.go +++ b/pkg/kv/kvserver/raft_log_truncator.go @@ -380,19 +380,10 @@ func (r rangesByRangeID) Swap(i, j int) { // deadlock (see storage.Engine.RegisterFlushCompletedCallback). func (t *raftLogTruncator) durabilityAdvancedCallback() { runTruncation := false - doneRunning := func() {} t.mu.Lock() if !t.mu.runningTruncation && len(t.mu.addRanges) > 0 { runTruncation = true t.mu.runningTruncation = true - doneRunning = func() { - t.mu.Lock() - defer t.mu.Unlock() - if !t.mu.runningTruncation { - panic("expected runningTruncation") - } - t.mu.runningTruncation = false - } } if !runTruncation && len(t.mu.addRanges) > 0 { t.mu.queuedDurabilityCB = true @@ -403,20 +394,31 @@ func (t *raftLogTruncator) durabilityAdvancedCallback() { } if err := t.stopper.RunAsyncTask(t.ambientCtx, "raft-log-truncation", func(ctx context.Context) { - defer doneRunning() for { t.durabilityAdvanced(ctx) + shouldReturn := false t.mu.Lock() queued := t.mu.queuedDurabilityCB t.mu.queuedDurabilityCB = false - t.mu.Unlock() if !queued { + t.mu.runningTruncation = false + shouldReturn = true + } + t.mu.Unlock() + if shouldReturn { return } } }); err != nil { // Task did not run because stopper is stopped. - doneRunning() + func() { + t.mu.Lock() + defer t.mu.Unlock() + if !t.mu.runningTruncation { + panic("expected runningTruncation") + } + t.mu.runningTruncation = false + }() } }