Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…db#77064 cockroachdb#77245

75285: spanconfig: introduce the ProtectedTSReader interface  r=adityamaru,nvanbenschoten,ajwerner a=arulajmani

See individual commits for details. 

Release justification: low risk, high benefit changes to existing
functionality.

76929: settings: Add syntax for cluster settings r=raduberinde,rafiss a=ajstorm

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 <id> SET CLUSTER SETTING <setting> = <value>
* ALTER TENANT ALL SET CLUSTER SETTING <setting> = <value>
* ALTER TENANT <id> RESET CLUSTER SETTING <setting>
* ALTER TENANT ALL RESET CLUSTER SETTING <setting>
* SHOW CLUSTER SETTING <setting> FOR TENANT <id>
* SHOW [ALL] CLUSTER SETTINGS FOR TENANT <id>

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.

Informs: cockroachdb#73857.

76943: Unary Complement execution has different results when the parameters are different r=otan a=ecwall

fixes cockroachdb#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.

77064: spanconfigkvsubscriber,kvserver: fix KVSubscriber bug r=arulajmani a=arulajmani

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

77245: kvserver: fix race in durability callback queueing in raftLogTruncator r=erikgrinaker a=sumeerbhola

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 cockroachdb#77046

Release justification: Bug fix

Release note: None

Co-authored-by: arulajmani <[email protected]>
Co-authored-by: Adam Storm <[email protected]>
Co-authored-by: Evan Wall <[email protected]>
Co-authored-by: sumeerbhola <[email protected]>
  • Loading branch information
5 people committed Mar 2, 2022
6 parents 34183fd + 365521a + 89091fe + 5eba198 + 9fbf507 + 3ba6dcb commit 1cc1725
Show file tree
Hide file tree
Showing 54 changed files with 1,282 additions and 976 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/alter_stmt.bnf
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
alter_stmt ::=
alter_ddl_stmt
| alter_role_stmt
| alter_tenant_csetting_stmt
5 changes: 5 additions & 0 deletions docs/generated/sql/bnf/alter_tenant_csetting_stmt.bnf
Original file line number Diff line number Diff line change
@@ -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
4 changes: 4 additions & 0 deletions docs/generated/sql/bnf/show_cluster_setting.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -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
49 changes: 30 additions & 19 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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

Expand Down Expand Up @@ -1671,10 +1694,6 @@ session_var ::=
| 'LC_CTYPE'
| 'TIME' 'ZONE'

var_name ::=
name
| name attrs

restore_options_list ::=
( restore_options ) ( ( ',' restore_options ) )*

Expand Down Expand Up @@ -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'
|
Expand Down Expand Up @@ -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'
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions pkg/gen/docs.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
5 changes: 1 addition & 4 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -336,10 +334,8 @@ 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/protectedts/ptverifier",
"//pkg/kv/kvserver/raftentry",
"//pkg/kv/kvserver/rditer",
"//pkg/kv/kvserver/readsummary/rspb",
Expand All @@ -360,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",
Expand Down
62 changes: 51 additions & 11 deletions pkg/kv/kvserver/client_protectedts_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"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/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"
Expand Down Expand Up @@ -210,12 +211,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.
ptsReader := tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().ProtectedTimestampReader
require.NoError(
t,
verifyProtectionTimestampExistsOnSpans(ctx, tc, ptsReader, ptsRec.Timestamp, ptsRec.DeprecatedSpans),
)

// Make a new record that is doomed to fail.
failedRec := ptsRec
Expand All @@ -226,17 +228,24 @@ 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)
// 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, ptsReader, failedRec.Timestamp, failedRec.DeprecatedSpans),
)

// Add a new record that is after the old record.
laterRec := ptsRec
laterRec.ID = uuid.MakeV4().GetBytes()
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, ptsReader, 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.
Expand All @@ -263,3 +272,34 @@ func TestProtectedTimestamps(t *testing.T) {
require.Len(t, state.Records, 0)
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,
ptsReader spanconfig.ProtectedTSReader,
protectionTimestamp hlc.Timestamp,
spans roachpb.Spans,
) error {
if err := spanconfigptsreader.TestingRefreshPTSState(
ctx, ptsReader, tc.Server(0).Clock().Now(),
); err != nil {
return err
}
for _, sp := range spans {
timestamps, _ := ptsReader.GetProtectionTimestamps(ctx, sp)
found := false
for _, ts := range timestamps {
if ts.Equal(protectionTimestamp) {
found = true
break
}
}
if !found {
return errors.Newf("protection timestamp %s does not exist on span %s", protectionTimestamp, sp)
}
}
return nil
}
34 changes: 25 additions & 9 deletions pkg/kv/kvserver/client_replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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")
}
}
Expand Down Expand Up @@ -3572,10 +3572,24 @@ 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)
}
}
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)
}
}
Expand Down Expand Up @@ -3632,13 +3646,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)
})
Expand Down
8 changes: 7 additions & 1 deletion pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
Loading

0 comments on commit 1cc1725

Please sign in to comment.