Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
93952: sql: measure CPU time spent during SQL execution r=DrewKimball a=DrewKimball

This commit adds tracking for CPU time spent during SQL execution. The CPU time is tracked at the operator granularity when statistics collection is enabled, similar to execution time.

For now, the CPU time is only surfaced in the output of `EXPLAIN ANALYZE` variants. A future PR will add support for logging this value in the statement statistics.

Informs: #87213

Release note (sql change): CPU time spent during SQL execution is now visible in the output of queries run with `EXPLAIN ANALYZE`. This measure does not include CPU time spent while serving KV requests. This can be useful for diagnosing performance issues and optimizing SQL queries.

95040: tenantcapabilities: introduce a Watcher over system.tenants r=knz a=arulajmani

This patch introduces three new interfaces -- a Watcher, a Reader,
and Authorizer. They're not hooked up yet, but once they are, they'll
work together to provide (in-memory) capability checks for incoming
tenant requests.

The Watcher establishes a rangefeed over `system.tenants` to
incrementally (and transparently) maintain an in-memory view of the
global tenant capability state. Publicly, it exposes a `Reader`
interface.

The `Reader` provides access to the global tenant capability state.
The `Watcher` and `Authorizer` communicate with each other using the
`Reader` interface.

The `Authorizer` consulsts the global tenant capability state to perform
authorization checks for incoming requests issued by tenants. Part of
the motivation to structure the code as such is to expand the set of
inputs the `Authorizer` uses to authorize requests. One could imagine
other dependencies being injected into the `Authorizer` in the future.

Epic: CRDB-18503
References: #94643

Release note: None

95361: sql/schemachanger: add compatibility with 22.2 rules r=fqazi a=fqazi

This PR will make the following changes:

1. Refactor the existing rules package so that there is a common package, and multiple dep/op rules registries to allow us to support rules from older releases. This also includes splitting out helper functions to make this process easier 
2. Import the 22.2 rules and have them adopt the same refactoring, so that a new rules/deps registry exists with them
3. Select the dep/op rules based on the active version of CRDB

Co-authored-by: Drew Kimball <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
Co-authored-by: Faizan Qazi <[email protected]>
  • Loading branch information
4 people committed Jan 19, 2023
4 parents 0bc78ed + d5f0481 + 1286dcc + 2644ed7 commit c8c4241
Show file tree
Hide file tree
Showing 98 changed files with 10,330 additions and 3,703 deletions.
3 changes: 2 additions & 1 deletion build/bazelutil/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -1547,7 +1547,8 @@
"cockroach/pkg/.*\\.eg\\.go$": "generated code",
".*\\.pb\\.go$": "generated code",
".*\\.pb\\.gw\\.go$": "generated code",
"cockroach/pkg/.*_generated\\.go$": "generated code"
"cockroach/pkg/.*_generated\\.go$": "generated code",
"cockroach/pkg/sql/schemachanger/scplan/internal/rules/.*/.*.go$": "schema changer rules"
},
"only_files": {
"cockroach/pkg/.*$": "first-party code"
Expand Down
16 changes: 14 additions & 2 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,7 @@ ALL_TESTS = [
"//pkg/kv/kvserver/uncertainty:uncertainty_test",
"//pkg/kv/kvserver:kvserver_test",
"//pkg/kv:kv_test",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:tenantcapabilitiesauthorizer_test",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher:tenantcapabilitieswatcher_test",
"//pkg/obsservice/obslib/ingest:ingest_test",
"//pkg/roachpb:roachpb_disallowed_imports_test",
Expand Down Expand Up @@ -474,7 +475,8 @@ ALL_TESTS = [
"//pkg/sql/schemachanger/scexec/backfiller:backfiller_test",
"//pkg/sql/schemachanger/scexec:scexec_test",
"//pkg/sql/schemachanger/scplan/internal/opgen:opgen_test",
"//pkg/sql/schemachanger/scplan/internal/rules:rules_test",
"//pkg/sql/schemachanger/scplan/internal/rules/current:current_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2_test",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test",
"//pkg/sql/schemachanger/scplan:scplan_test",
"//pkg/sql/schemachanger/screl:screl_test",
Expand Down Expand Up @@ -1269,7 +1271,10 @@ GO_TARGETS = [
"//pkg/kv:kv_test",
"//pkg/multitenant/multitenantcpu:multitenantcpu",
"//pkg/multitenant/multitenantio:multitenantio",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:tenantcapabilitiesauthorizer",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:tenantcapabilitiesauthorizer_test",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils:tenantcapabilitiestestutils",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher:tenantcapabilitieswatcher",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher:tenantcapabilitieswatcher_test",
"//pkg/multitenant/tenantcapabilities:tenantcapabilities",
Expand Down Expand Up @@ -1773,8 +1778,11 @@ GO_TARGETS = [
"//pkg/sql/schemachanger/scpb:scpb",
"//pkg/sql/schemachanger/scplan/internal/opgen:opgen",
"//pkg/sql/schemachanger/scplan/internal/opgen:opgen_test",
"//pkg/sql/schemachanger/scplan/internal/rules/current:current",
"//pkg/sql/schemachanger/scplan/internal/rules/current:current_test",
"//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2",
"//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:release_22_2_test",
"//pkg/sql/schemachanger/scplan/internal/rules:rules",
"//pkg/sql/schemachanger/scplan/internal/rules:rules_test",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph",
"//pkg/sql/schemachanger/scplan/internal/scgraph:scgraph_test",
"//pkg/sql/schemachanger/scplan/internal/scgraphviz:scgraphviz",
Expand Down Expand Up @@ -2613,7 +2621,9 @@ GET_X_DATA_TARGETS = [
"//pkg/multitenant/multitenantcpu:get_x_data",
"//pkg/multitenant/multitenantio:get_x_data",
"//pkg/multitenant/tenantcapabilities:get_x_data",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:get_x_data",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:get_x_data",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils:get_x_data",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher:get_x_data",
"//pkg/multitenant/tenantcostmodel:get_x_data",
"//pkg/obs:get_x_data",
Expand Down Expand Up @@ -2923,6 +2933,8 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/schemachanger/scplan:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/opgen:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/rules:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/rules/current:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/rules/release_22_2:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/scgraph:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/scgraphviz:get_x_data",
"//pkg/sql/schemachanger/scplan/internal/scstage:get_x_data",
Expand Down
7 changes: 7 additions & 0 deletions pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,13 @@ const (
IncrementalUpdate UpdateType = false
)

func (u UpdateType) String() string {
if u == CompleteUpdate {
return "Complete Update"
}
return "Incremental Update"
}

// TranslateEventFunc is used by the client to translate a low-level event
// into an event for buffering. If nil is returned, the event is skipped.
type TranslateEventFunc func(
Expand Down
7 changes: 6 additions & 1 deletion pkg/multitenant/tenantcapabilities/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,10 +3,15 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")

go_library(
name = "tenantcapabilities",
srcs = ["capabilities.go"],
srcs = [
"capabilities.go",
"testingknobs.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities",
visibility = ["//visibility:public"],
deps = [
"//pkg/base",
"//pkg/kv/kvclient/rangefeed/rangefeedcache",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb",
"//pkg/roachpb",
],
Expand Down
42 changes: 40 additions & 2 deletions pkg/multitenant/tenantcapabilities/capabilities.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,12 +11,50 @@
package tenantcapabilities

import (
"context"

"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

// CapabilitiesEntry ties together a tenantID with its capabilities.
type CapabilitiesEntry struct {
// Watcher presents a consistent snapshot of the global tenant capabilities
// state. It incrementally, and transparently, maintains this state by watching
// for changes to system.tenants.
type Watcher interface {
Reader

// Start asynchronously begins watching over the global tenant capability
// state.
Start(ctx context.Context) error
}

// Reader provides access to the global tenant capability state. The global
// tenant capability state may be arbitrarily stale.
type Reader interface {
GetCapabilities(id roachpb.TenantID) (_ tenantcapabilitiespb.TenantCapabilities, found bool)
}

// Authorizer performs various kinds of capability checks for requests issued
// by tenants. It does so by consulting the global tenant capability state.
//
// In the future, we may want to expand the Authorizer to take into account
// signals other than just the tenant capability state. For example, request
// usage pattern over a timespan.
type Authorizer interface {
// HasCapabilityForBatch returns whether a tenant, referenced by its ID, is
// allowed to execute the supplied batch request given the capabilities it
// possesses.
HasCapabilityForBatch(context.Context, roachpb.TenantID, *roachpb.BatchRequest) bool
}

// Entry ties together a tenantID with its capabilities.
type Entry struct {
TenantID roachpb.TenantID
TenantCapabilities tenantcapabilitiespb.TenantCapabilities
}

// Update represents an update to the global tenant capability state.
type Update struct {
Entry
Deleted bool // whether the entry was deleted or not
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "tenantcapabilitiesauthorizer",
srcs = ["authorizer.go"],
importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer",
visibility = ["//visibility:public"],
deps = [
"//pkg/multitenant/tenantcapabilities",
"//pkg/roachpb",
"//pkg/util/log",
],
)

go_test(
name = "tenantcapabilitiesauthorizer_test",
srcs = ["authorizer_test.go"],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":tenantcapabilitiesauthorizer"],
deps = [
"//pkg/multitenant/tenantcapabilities",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils",
"//pkg/roachpb",
"//pkg/testutils/datapathutils",
"//pkg/util/leaktest",
"@com_github_cockroachdb_datadriven//:datadriven",
],
)

get_x_data(name = "get_x_data")
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
// Copyright 2023 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 tenantcapabilitiesauthorizer

import (
"context"

"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

// Authorizer is a concrete implementation of the tenantcapabilities.Authorizer
// interface. It's safe for concurrent use.
type Authorizer struct {
capabilitiesReader tenantcapabilities.Reader
}

var _ tenantcapabilities.Authorizer = &Authorizer{}

// New constructs a new tenantcapabilities.Authorizer.
func New(reader tenantcapabilities.Reader) *Authorizer {
a := &Authorizer{
capabilitiesReader: reader,
}
return a
}

// HasCapabilityForBatch implements the tenantcapabilities.Authorizer interface.
func (a *Authorizer) HasCapabilityForBatch(
ctx context.Context, tenID roachpb.TenantID, ba *roachpb.BatchRequest,
) bool {
if tenID.IsSystem() {
return true // the system tenant is allowed to do as it pleases
}
cp, found := a.capabilitiesReader.GetCapabilities(tenID)
if !found {
log.Infof(
ctx,
"no capability information for tenant %s; requests that require capabilities may be denied",
tenID,
)
}

for _, ru := range ba.Requests {
switch ru.GetInner().(type) {
case *roachpb.AdminSplitRequest:
if !cp.CanAdminSplit {
return false
}
default:
// No capability checks for other types of requests.
}
}
return true
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
// Copyright 2023 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 tenantcapabilitiesauthorizer

import (
"context"
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/datadriven"
)

// TestDataDriven runs datadriven tests against the Authorizer interface. The
// syntax is as follows:
//
// "update-state": updates the underlying global tenant capability state.
// Example:
//
// update-state
// upsert {ten=10}:{CanAdminSplit=true}
// delete {ten=15}
// ----
//
// "has-capability-for-batch": performs a capability check, given a tenant and
// batch request declaration. Example:
//
// has-capability-for-batch
// {ten=10}
// split
// ----
func TestDataDriven(t *testing.T) {
defer leaktest.AfterTest(t)()

datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) {
mockReader := mockReader(make(map[roachpb.TenantID]tenantcapabilitiespb.TenantCapabilities))
authorizer := New(mockReader)

datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string {
switch d.Cmd {
case "update-state":
updates := tenantcapabilitiestestutils.ParseTenantCapabilityUpdateStateArguments(t, d.Input)
mockReader.updateState(updates)

case "has-capability-for-batch":
tenID, ba := tenantcapabilitiestestutils.ParseBatchRequestString(t, d.Input)
hasCapability := authorizer.HasCapabilityForBatch(context.Background(), tenID, &ba)
return fmt.Sprintf("%t", hasCapability)

default:
return fmt.Sprintf("unknown command %s", d.Cmd)
}
return "ok"
})
})
}

type mockReader map[roachpb.TenantID]tenantcapabilitiespb.TenantCapabilities

func (m mockReader) updateState(updates []tenantcapabilities.Update) {
for _, update := range updates {
if update.Deleted {
delete(m, update.TenantID)
} else {
m[update.TenantID] = update.TenantCapabilities
}
}
}

// GetCapabilities implements the tenantcapabilities.Reader interface.
func (m mockReader) GetCapabilities(
id roachpb.TenantID,
) (tenantcapabilitiespb.TenantCapabilities, bool) {
cp, found := m[id]
return cp, found
}
Loading

0 comments on commit c8c4241

Please sign in to comment.