Skip to content

Commit

Permalink
Merge pull request #101163 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-23.1-100921

release-23.1: multitenant: add can_check_consistency capability
  • Loading branch information
ecwall authored Apr 11, 2023
2 parents d5b5674 + 0650230 commit 64903e1
Show file tree
Hide file tree
Showing 20 changed files with 210 additions and 123 deletions.
11 changes: 11 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/tenant_capability
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -59,6 +60,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -76,6 +78,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -100,6 +103,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -124,6 +128,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -148,6 +153,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info true
can_view_tsdb_metrics false
Expand All @@ -165,6 +171,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -182,6 +189,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand All @@ -199,6 +207,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split false
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand Down Expand Up @@ -236,6 +245,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand Down Expand Up @@ -264,6 +274,7 @@ can_admin_relocate_range false
can_admin_scatter true
can_admin_split true
can_admin_unsplit false
can_check_consistency false
can_use_nodelocal_storage false
can_view_node_info false
can_view_tsdb_metrics false
Expand Down
4 changes: 4 additions & 0 deletions pkg/multitenant/tenantcapabilities/capabilities.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,9 @@ const (
// want to hide from customer tenants in CockroachCloud.
CanViewNodeInfo // can_view_node_info

// CanCheckConsistency allows the tenant to check range consistency.
CanCheckConsistency // can_check_consistency

// CanViewTSDBMetrics describes the ability of a tenant to read the
// timeseries from the storage cluster. These operations need a
// capability because excessive TS queries can overwhelm the storage
Expand Down Expand Up @@ -116,6 +119,7 @@ var capabilities = [MaxCapabilityID + 1]Capability{
CanAdminScatter: boolCapability(CanAdminScatter),
CanAdminSplit: boolCapability(CanAdminSplit),
CanAdminUnsplit: boolCapability(CanAdminUnsplit),
CanCheckConsistency: boolCapability(CanCheckConsistency),
CanUseNodelocalStorage: boolCapability(CanUseNodelocalStorage),
CanViewNodeInfo: boolCapability(CanViewNodeInfo),
CanViewTSDBMetrics: boolCapability(CanViewTSDBMetrics),
Expand Down
13 changes: 7 additions & 6 deletions pkg/multitenant/tenantcapabilities/id_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Original file line number Diff line number Diff line change
Expand Up @@ -159,12 +159,12 @@ var reqMethodToCap = map[kvpb.Method]tenantcapabilities.ID{
kvpb.AdminUnsplit: tenantcapabilities.CanAdminUnsplit,
kvpb.AdminRelocateRange: tenantcapabilities.CanAdminRelocateRange,
kvpb.AdminTransferLease: tenantcapabilities.CanAdminRelocateRange,
kvpb.CheckConsistency: tenantcapabilities.CanCheckConsistency,

// TODO(knz,arul): Verify with the relevant teams whether secondary
// tenants have legitimate access to any of those.
kvpb.AdminMerge: onlySystemTenant,
kvpb.AdminVerifyProtectedTimestamp: onlySystemTenant,
kvpb.CheckConsistency: onlySystemTenant,
kvpb.ComputeChecksum: onlySystemTenant,
kvpb.GC: onlySystemTenant,
kvpb.Merge: onlySystemTenant,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,10 @@ message TenantCapabilities {
// CanUseNodelocalStorage if set to true, grants the tenant the ability
// to use nodelocal storage on any KV node.
bool can_use_nodelocal_storage = 9;

// CanCheckConsistency if set to true, grants the tenant the ability to run
// range consistency checks.
bool can_check_consistency = 10;
};

// SpanConfigBound is used to constrain the possible values a SpanConfig may
Expand Down
2 changes: 2 additions & 0 deletions pkg/multitenant/tenantcapabilities/values.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@ func GetValueByID(t *tenantcapabilitiespb.TenantCapabilities, id ID) (Value, err
return (*invertedBoolValue)(&t.DisableAdminSplit), nil
case CanAdminUnsplit:
return (*boolValue)(&t.CanAdminUnsplit), nil
case CanCheckConsistency:
return (*boolValue)(&t.CanCheckConsistency), nil
case CanUseNodelocalStorage:
return (*boolValue)(&t.CanUseNodelocalStorage), nil
case CanViewNodeInfo:
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/faketreeeval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/util/duration",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/mon",
"//pkg/util/rangedesc",
"@com_github_cockroachdb_errors//:errors",
"@com_github_lib_pq//oid",
],
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/faketreeeval/evalctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/rangedesc"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
Expand Down Expand Up @@ -468,6 +469,13 @@ func (ep *DummyEvalPlanner) EnforceHomeRegion() bool {
return false
}

// GetRangeDescIterator is part of the eval.Planner interface.
func (ep *DummyEvalPlanner) GetRangeDescIterator(
context.Context, roachpb.Span,
) (_ rangedesc.Iterator, _ error) {
return
}

// GetRangeDescByID is part of the eval.Planner interface.
func (ep *DummyEvalPlanner) GetRangeDescByID(
context.Context, roachpb.RangeID,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ go_library(
"//pkg/kv/kvclient/rangefeed",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/kvserverbase",
"//pkg/multitenant/tenantcapabilities",
"//pkg/security/username",
"//pkg/server",
"//pkg/settings/cluster",
Expand Down
22 changes: 18 additions & 4 deletions pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -1536,20 +1537,33 @@ func (t *logicTest) newCluster(
}
}

tenantID := serverutils.TestTenantID().ToUint64()
tenantID := serverutils.TestTenantID()
for name, value := range clusterSettings {
query := fmt.Sprintf("ALTER TENANT [$1] SET CLUSTER SETTING %s = $2", name)
if _, err := conn.Exec(query, tenantID, value); err != nil {
if _, err := conn.Exec(query, tenantID.ToUint64(), value); err != nil {
t.Fatal(err)
}
}

for name, value := range toa.capabilities {
capabilities := toa.capabilities
for name, value := range capabilities {
query := fmt.Sprintf("ALTER TENANT [$1] GRANT CAPABILITY %s = $2", name)
if _, err := conn.Exec(query, tenantID, value); err != nil {
if _, err := conn.Exec(query, tenantID.ToUint64(), value); err != nil {
t.Fatal(err)
}
}
numCapabilities := len(capabilities)
if numCapabilities > 0 {
capabilityMap := make(map[tenantcapabilities.ID]string, numCapabilities)
for k, v := range capabilities {
capability, ok := tenantcapabilities.FromName(k)
if !ok {
t.Fatalf("cannot get capability from name %q", k)
}
capabilityMap[capability.ID()] = v
}
t.cluster.WaitForTenantCapabilities(t.t(), tenantID, capabilityMap)
}
}

var randomWorkmem int
Expand Down
33 changes: 32 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/builtin_function
Original file line number Diff line number Diff line change
@@ -1,3 +1,5 @@
# tenant-capability-override-opt: can_check_consistency=true

statement ok
CREATE TABLE foo (a int)

Expand Down Expand Up @@ -3869,7 +3871,8 @@ SELECT ‹'yes'› IN (‹'no'›, ‹'maybe'›, ‹'yes'›)
query T
SELECT crdb_internal.redactable_sql_constants(e'\r')
----
‹›

query T
SELECT crdb_internal.redactable_sql_constants('SELECT crdb_internal.redactable_sql_constants('''')')
Expand Down Expand Up @@ -3973,3 +3976,31 @@ query T
SELECT crdb_internal.redact(crdb_internal.redactable_sql_constants(ARRAY['SELECT 1', NULL, 'SELECT ''hello''', '']))
----
{"SELECT ‹×›",NULL,"SELECT ‹×›",‹×›}

subtest check_consistency

# Sanity-check crdb_internal.check_consistency.

statement error start key must be >=
SELECT crdb_internal.check_consistency(true, '\x00', crdb_internal.tenant_span()[2])

statement error end key must be <=
SELECT crdb_internal.check_consistency(true, crdb_internal.tenant_span()[1], '\xffff00')

statement error start key must be less than end key
SELECT crdb_internal.check_consistency(true, crdb_internal.tenant_span()[2], crdb_internal.tenant_span()[2])

query TT
SELECT status, regexp_replace(detail, '[0-9]+', '', 'g')
FROM crdb_internal.check_consistency(true, crdb_internal.tenant_span()[1], crdb_internal.tenant_span()[2])
ORDER BY range_id
LIMIT 1
----
RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: IntentAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: SeparatedIntentCount: RangeKeyCount: RangeKeyBytes: RangeValCount: RangeValBytes: SysBytes: SysCount: AbortSpanBytes:}

# Fill a table with consistency check results. This used to panic.
# See: https://github.com/cockroachdb/cockroach/issues/88222
statement ok
CREATE TABLE conscheckresult AS (SELECT * FROM crdb_internal.check_consistency(false, '', ''));

subtest end
28 changes: 2 additions & 26 deletions pkg/sql/logictest/testdata/logic_test/builtin_function_notenant
Original file line number Diff line number Diff line change
Expand Up @@ -2,25 +2,6 @@

subtest check_consistency

# Sanity-check crdb_internal.check_consistency.

statement error start key must be > "\\x02"
SELECT crdb_internal.check_consistency(true, '\x02', '\xffff')

statement error end key must be < "\\xff\\xff"
SELECT crdb_internal.check_consistency(true, '\x0200', '\xffff00')

statement error start key must be less than end key
SELECT crdb_internal.check_consistency(true, '\x03', '\x03')

statement error start key must be less than end key
SELECT crdb_internal.check_consistency(true, '\x04', '\x03')

query ITT
SELECT range_id, status, regexp_replace(detail, '[0-9]+', '', 'g') FROM crdb_internal.check_consistency(true, '\x03', '\xffff') WHERE range_id = 1
----
1 RANGE_CONSISTENT stats: {ContainsEstimates: LastUpdateNanos: IntentAge: GCBytesAge: LiveBytes: LiveCount: KeyBytes: KeyCount: ValBytes: ValCount: IntentBytes: IntentCount: SeparatedIntentCount: RangeKeyCount: RangeKeyBytes: RangeValCount: RangeValBytes: SysBytes: SysCount: AbortSpanBytes:}

# Without explicit keys, scans all ranges (we don't test this too precisely to
# avoid flaking the test when the range count changes, just want to know that
# we're touching multiple ranges).
Expand Down Expand Up @@ -51,17 +32,12 @@ SELECT count(*) = 1 FROM crdb_internal.check_consistency(true, '\xff', '')
----
true

# Run a full consistency check across all ranges.
# Run a full consistency check across all ranges.
query B
SELECT count(*) > 10 FROM crdb_internal.check_consistency(false, '', '')
----
true

# Fill a table with consistency check results. This used to panic.
# See: https://github.com/cockroachdb/cockroach/issues/88222
statement ok
CREATE TABLE conscheckresult AS (SELECT * FROM crdb_internal.check_consistency(false, '', ''));

# Test crdb_internal commands which execute as root, but
# only checks for permissions afterwards.
subtest crdb_internal_privileged_only
Expand Down Expand Up @@ -158,7 +134,7 @@ SELECT crdb_internal.get_zone_config(crdb_internal.get_namespace_id(0, 'root_tes
query T
SELECT ($t_id)::regclass
----
109
108

# reset state to default
user root
Expand Down
Loading

0 comments on commit 64903e1

Please sign in to comment.