Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
93153: rttanalysis: don't count leasing the database desc r=andreimatei a=andreimatei

A bunch of rtt-analysis tests were counting a request for leasing the database descriptor. This is not interesting. This patch makes the test framework lease it first through a "USE" statement.

The number of KV requests required for leasing is currently mis-counted. We count 1, but in reality it's 4. A different patch will correct the miscounting that, at which point that would be too significant for the tests.

Release note: None
Epic: None

93325: multitenant: retain range splits after TRUNCATE for secondary tenants r=knz a=ecwall

Fixes #69499
Fixes #82944

Existing split points are preserved after a TRUNCATE statement is executed by a secondary tenant.

Release note: None

93354: tracing: disallow children of sterile span with different Tracer r=andreimatei a=andreimatei

Before this patch, creating a "child" of a sterile span with a different Tracer than the one used to create the sterile span was tolerated - on the argument that sterile spans don't actually get children (the would-be child span is created as a root), so the arguments for not allowing a children to be created with different tracers don't apply. At the same time, creating a child of a noop span with a different Tracer than the noop span's Tracer was documented to not be allowed. In practice, it was, because the code was confused [1].

This patch disallows creating children of sterile spans with a different tracer, for consistency with all the other spans. The patch also makes it a panic for the children of noop spans to be created with a different Tracer.

This is all meant as a cleanup / code simplification.

[1] WithParent(sp) meant to treat sterile spans differently than noop spans but it was using sp.IsSterile(), which returns true for both. As such, it was unintentionally returning an empty parent option. startSpanGeneric() meant to check the tracer of parent noop spans, but it was failing to actually do so because it was going through the opts.Parent.empty().

Release note: None
Epic: None

93545: sql: make SHOW RANGES FOR TABLE include all indexes r=ajwerner a=knz

Informs #80906.
Fixes #93546.
Fixes #82995.

Release note (backward-incompatible change): `SHOW RANGES FOR TABLE`
now includes rows for all indexes that support the table. Prior to
this change, `SHOW RANGES FOR TABLE foo` was an alias for `SHOW RANGES
FOR INDEX foo@primary`. This was causing confusion, as it would miss
data for secondary indexes. It is still possible to filter to just the
primary index using `SHOW RANGES FOR INDEX foo@primary`.

The statement output now also includes the index name.

93557: syntheticprivilegecache: scan all privileges at startup  r=ajwerner a=ajwerner

#### syntheticprivilegecache: move caching logic out of sql
This is a pure refactor to move the logic for caching synthetic privileges
from the sql package. This will make it easier to add features later.

#### syntheticprivilegecache: scan all privileges at startup 


Fixes #93182

This commit attempts to alleviate the pain caused by synthetic virtual table
privileges introduced in 22.2. We need to fetch privileges for virtual tables
to determine whether the user has access. This is done lazily. However, when a
user attempts to read a virtual table like pg_class or run SHOW TABLES it will
force the privileges to be determined for each virtual table (of which there
are 290 at the time of writing). This sequential process can be somewhat slow
in a single region cluster and will be very slow in an MR cluster.

This patch attempts to somewhat alleviate this pain by scanning the table
eagerly during server startup.

Release note (performance improvement): In 22.2 we introduced privileges on
virtual tables (system catalogs like pg_catalog, information_schema, and
crdb_internal). A problem with this new feature is that we now must fetch those
privileges into a cache before we can use those tables or determine their
visibility in other system catalogs. This process used to occur on-demand, when
the privilege was needed. Now we'll fetch these privileges eagerly during
startup to mitigate the latency when accessing pg_catalog right after the
server boots up.

93563: pgwire: handle decoding Geometry/Geography in binary r=rafiss a=otan

Resolves #81066
Resolves #93352

Release note (bug fix): Previously, CockroachDB would error when receiving Geometry/Geography using binary parameters. This is now resolved.

93618: cli,cliccl: move some mt commands to cliccl r=ajwerner a=ajwerner

Part of #91714

Epic: none

Release note: None

Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Evan Wall <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
7 people committed Dec 14, 2022
8 parents ebcf24c + 21c408f + 7287323 + e30010f + 5604fea + 268d0f4 + 6ed8b8a + 3bde92a commit a30fb14
Show file tree
Hide file tree
Showing 56 changed files with 981 additions and 431 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1837,6 +1837,7 @@ GO_TARGETS = [
"//pkg/sql/storageparam:storageparam",
"//pkg/sql/syntheticprivilege:syntheticprivilege",
"//pkg/sql/syntheticprivilege:syntheticprivilege_test",
"//pkg/sql/syntheticprivilegecache:syntheticprivilegecache",
"//pkg/sql/tests:tests",
"//pkg/sql/tests:tests_test",
"//pkg/sql/ttl/ttlbase:ttlbase",
Expand Down Expand Up @@ -2931,6 +2932,7 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/storageparam/indexstorageparam:get_x_data",
"//pkg/sql/storageparam/tablestorageparam:get_x_data",
"//pkg/sql/syntheticprivilege:get_x_data",
"//pkg/sql/syntheticprivilegecache:get_x_data",
"//pkg/sql/tests:get_x_data",
"//pkg/sql/ttl/ttlbase:get_x_data",
"//pkg/sql/ttl/ttljob:get_x_data",
Expand Down
10 changes: 9 additions & 1 deletion pkg/bench/rttanalysis/rtt_analysis_bench.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,15 @@ func executeRoundTripTest(b testingB, tc RoundTripBenchTestCase, cc ClusterConst
// Do an extra iteration and don't record it in order to deal with effects of
// running it the first time.
for i := 0; i < b.N()+1; i++ {
sql.Exec(b, "CREATE DATABASE bench;")
sql.Exec(b, "CREATE DATABASE bench")
// Make sure the database descriptor is leased, so that tests don't count
// the leasing.
sql.Exec(b, "USE bench")
// Also force a lease on the "public" schema too.
sql.Exec(b, "CREATE TABLE bench.public.__dummy__()")
sql.Exec(b, "SELECT 1 FROM bench.public.__dummy__")
sql.Exec(b, "DROP TABLE bench.public.__dummy__")

sql.Exec(b, tc.Setup)
for _, statement := range statements {
cluster.clearStatementTrace(statement.SQL)
Expand Down
76 changes: 38 additions & 38 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -27,10 +27,10 @@ exp,benchmark
7,AlterTableUnsplit/alter_table_unsplit_at_1_value
9,AlterTableUnsplit/alter_table_unsplit_at_2_values
11,AlterTableUnsplit/alter_table_unsplit_at_3_values
13,CreateRole/create_role_with_1_option
16,CreateRole/create_role_with_2_options
19,CreateRole/create_role_with_3_options
14,CreateRole/create_role_with_no_options
12,CreateRole/create_role_with_1_option
15,CreateRole/create_role_with_2_options
18,CreateRole/create_role_with_3_options
13,CreateRole/create_role_with_no_options
10,DropDatabase/drop_database_0_tables
11,DropDatabase/drop_database_1_table
11,DropDatabase/drop_database_2_tables
Expand All @@ -47,46 +47,46 @@ exp,benchmark
12,DropView/drop_1_view
13,DropView/drop_2_views
13,DropView/drop_3_views
10,Grant/grant_all_on_1_table
10,Grant/grant_all_on_2_tables
10,Grant/grant_all_on_3_tables
9,Grant/grant_all_on_1_table
9,Grant/grant_all_on_2_tables
9,Grant/grant_all_on_3_tables
11,GrantRole/grant_1_role
15,GrantRole/grant_2_roles
4,ORMQueries/activerecord_type_introspection_query
6,ORMQueries/django_table_introspection_1_table
6,ORMQueries/django_table_introspection_4_tables
6,ORMQueries/django_table_introspection_8_tables
2,ORMQueries/has_column_privilege_using_attnum
2,ORMQueries/has_column_privilege_using_column_name
1,ORMQueries/has_schema_privilege_1
1,ORMQueries/has_schema_privilege_3
1,ORMQueries/has_schema_privilege_5
2,ORMQueries/has_sequence_privilege_1
4,ORMQueries/has_sequence_privilege_3
6,ORMQueries/has_sequence_privilege_5
2,ORMQueries/has_table_privilege_1
4,ORMQueries/has_table_privilege_3
6,ORMQueries/has_table_privilege_5
3,ORMQueries/activerecord_type_introspection_query
5,ORMQueries/django_table_introspection_1_table
5,ORMQueries/django_table_introspection_4_tables
5,ORMQueries/django_table_introspection_8_tables
1,ORMQueries/has_column_privilege_using_attnum
1,ORMQueries/has_column_privilege_using_column_name
0,ORMQueries/has_schema_privilege_1
0,ORMQueries/has_schema_privilege_3
0,ORMQueries/has_schema_privilege_5
1,ORMQueries/has_sequence_privilege_1
3,ORMQueries/has_sequence_privilege_3
5,ORMQueries/has_sequence_privilege_5
1,ORMQueries/has_table_privilege_1
3,ORMQueries/has_table_privilege_3
5,ORMQueries/has_table_privilege_5
85,ORMQueries/hasura_column_descriptions
85,ORMQueries/hasura_column_descriptions_8_tables
6,ORMQueries/hasura_column_descriptions_modified
6,ORMQueries/information_schema._pg_index_position
5,ORMQueries/pg_attribute
5,ORMQueries/pg_class
7,ORMQueries/pg_is_other_temp_schema
7,ORMQueries/pg_is_other_temp_schema_multiple_times
4,ORMQueries/pg_my_temp_schema
4,ORMQueries/pg_my_temp_schema_multiple_times
4,ORMQueries/pg_namespace
5,ORMQueries/pg_type
10,Revoke/revoke_all_on_1_table
10,Revoke/revoke_all_on_2_tables
10,Revoke/revoke_all_on_3_tables
5,ORMQueries/hasura_column_descriptions_modified
5,ORMQueries/information_schema._pg_index_position
4,ORMQueries/pg_attribute
4,ORMQueries/pg_class
6,ORMQueries/pg_is_other_temp_schema
6,ORMQueries/pg_is_other_temp_schema_multiple_times
3,ORMQueries/pg_my_temp_schema
3,ORMQueries/pg_my_temp_schema_multiple_times
3,ORMQueries/pg_namespace
4,ORMQueries/pg_type
9,Revoke/revoke_all_on_1_table
9,Revoke/revoke_all_on_2_tables
9,Revoke/revoke_all_on_3_tables
9,RevokeRole/revoke_1_role
11,RevokeRole/revoke_2_roles
1,SystemDatabaseQueries/select_system.users_with_empty_database_Name
1,SystemDatabaseQueries/select_system.users_with_schema_Name
2,SystemDatabaseQueries/select_system.users_without_schema_Name
1,SystemDatabaseQueries/select_system.users_without_schema_Name
12,Truncate/truncate_1_column_0_rows
12,Truncate/truncate_1_column_1_row
12,Truncate/truncate_1_column_2_rows
Expand All @@ -95,5 +95,5 @@ exp,benchmark
12,Truncate/truncate_2_column_2_rows
1,VirtualTableQueries/select_crdb_internal.invalid_objects_with_1_fk
1,VirtualTableQueries/select_crdb_internal.tables_with_1_fk
4,VirtualTableQueries/virtual_table_cache_with_point_lookups
8,VirtualTableQueries/virtual_table_cache_with_schema_change
3,VirtualTableQueries/virtual_table_cache_with_point_lookups
7,VirtualTableQueries/virtual_table_cache_with_schema_change
18 changes: 14 additions & 4 deletions pkg/bench/rttanalysis/validate_benchmark_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@ var (
rewriteIterations = flag.Int("rewrite-iterations", 50,
"if re-writing, the number of times to execute each benchmark to "+
"determine the range of possible values")
allowOffByOne = flag.Bool("allow-off-by-one", true,
"if set, expectations that are not a range get a ±1 tolerance")
)

// RunBenchmarkExpectationTests runs tests to validate or rewrite the contents
Expand Down Expand Up @@ -319,10 +321,18 @@ func (b benchmarkExpectations) find(name string) (benchmarkExpectation, bool) {
}

func (e benchmarkExpectation) matches(roundTrips int) bool {
// Either the value falls within the expected range, or
return (e.min <= roundTrips && roundTrips <= e.max) ||
// the expectation isn't a range, so give it a leeway of one.
e.min == e.max && (roundTrips == e.min-1 || roundTrips == e.min+1)
// Does the value fall in the range?
if e.min <= roundTrips && roundTrips <= e.max {
return true
}

// If the expectation isn't a range, it gets a leeway of one because we got
// tired of small indeterminism.
if (e.min == e.max) && *allowOffByOne && (roundTrips == e.min-1 || roundTrips == e.min+1) {
return true
}

return false
}

func (e benchmarkExpectation) String() string {
Expand Down
11 changes: 11 additions & 0 deletions pkg/ccl/cliccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,9 +5,14 @@ go_library(
name = "cliccl",
srcs = [
"cliccl.go",
"context.go",
"debug.go",
"demo.go",
"ear.go",
"flags.go",
"mt.go",
"mt_proxy.go",
"mt_test_directory.go",
"start.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/cliccl",
Expand All @@ -16,21 +21,27 @@ go_library(
"//pkg/base",
"//pkg/ccl/baseccl",
"//pkg/ccl/cliccl/cliflagsccl",
"//pkg/ccl/sqlproxyccl",
"//pkg/ccl/sqlproxyccl/tenantdirsvr",
"//pkg/ccl/storageccl/engineccl/enginepbccl",
"//pkg/ccl/utilccl",
"//pkg/ccl/workloadccl/cliccl",
"//pkg/cli",
"//pkg/cli/clierrorplus",
"//pkg/cli/cliflagcfg",
"//pkg/cli/cliflags",
"//pkg/cli/democluster",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/util/log",
"//pkg/util/log/severity",
"//pkg/util/protoutil",
"//pkg/util/stop",
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_cockroachdb_redact//:redact",
"@com_github_spf13_cobra//:cobra",
],
)
Expand Down
51 changes: 51 additions & 0 deletions pkg/ccl/cliccl/context.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package cliccl

import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl"
)

func init() {
setProxyContextDefaults()
setTestDirectorySvrContextDefaults()
}

// proxyContext captures the command-line parameters of the `mt start-proxy` command.
var proxyContext sqlproxyccl.ProxyOptions

func setProxyContextDefaults() {
proxyContext.Denylist = ""
proxyContext.ListenAddr = "127.0.0.1:46257"
proxyContext.ListenCert = ""
proxyContext.ListenKey = ""
proxyContext.MetricsAddress = "0.0.0.0:8080"
proxyContext.RoutingRule = ""
proxyContext.DirectoryAddr = ""
proxyContext.SkipVerify = false
proxyContext.Insecure = false
proxyContext.RatelimitBaseDelay = 50 * time.Millisecond
proxyContext.ValidateAccessInterval = 30 * time.Second
proxyContext.PollConfigInterval = 30 * time.Second
proxyContext.ThrottleBaseDelay = time.Second
proxyContext.DisableConnectionRebalancing = false
}

var testDirectorySvrContext struct {
port int
certsDir string
kvAddrs string
tenantBaseDir string
}

func setTestDirectorySvrContextDefaults() {
testDirectorySvrContext.port = 36257
}
45 changes: 45 additions & 0 deletions pkg/ccl/cliccl/flags.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package cliccl

import (
"github.com/cockroachdb/cockroach/pkg/cli"
"github.com/cockroachdb/cockroach/pkg/cli/cliflagcfg"
"github.com/cockroachdb/cockroach/pkg/cli/cliflags"
)

func init() {
// Multi-tenancy proxy command flags.
{
f := mtStartSQLProxyCmd.Flags()
cliflagcfg.StringFlag(f, &proxyContext.Denylist, cliflags.DenyList)
cliflagcfg.StringFlag(f, &proxyContext.ListenAddr, cliflags.ProxyListenAddr)
cliflagcfg.StringFlag(f, &proxyContext.ListenCert, cliflags.ListenCert)
cliflagcfg.StringFlag(f, &proxyContext.ListenKey, cliflags.ListenKey)
cliflagcfg.StringFlag(f, &proxyContext.MetricsAddress, cliflags.ListenMetrics)
cliflagcfg.StringFlag(f, &proxyContext.RoutingRule, cliflags.RoutingRule)
cliflagcfg.StringFlag(f, &proxyContext.DirectoryAddr, cliflags.DirectoryAddr)
cliflagcfg.BoolFlag(f, &proxyContext.SkipVerify, cliflags.SkipVerify)
cliflagcfg.BoolFlag(f, &proxyContext.Insecure, cliflags.InsecureBackend)
cliflagcfg.DurationFlag(f, &proxyContext.ValidateAccessInterval, cliflags.ValidateAccessInterval)
cliflagcfg.DurationFlag(f, &proxyContext.PollConfigInterval, cliflags.PollConfigInterval)
cliflagcfg.DurationFlag(f, &proxyContext.ThrottleBaseDelay, cliflags.ThrottleBaseDelay)
cliflagcfg.BoolFlag(f, &proxyContext.DisableConnectionRebalancing, cliflags.DisableConnectionRebalancing)
}

// Multi-tenancy test directory command flags.
cli.RegisterFlags(func() {
f := mtTestDirectorySvr.Flags()
cliflagcfg.IntFlag(f, &testDirectorySvrContext.port, cliflags.TestDirectoryListenPort)
cliflagcfg.StringFlag(f, &testDirectorySvrContext.certsDir, cliflags.TestDirectoryTenantCertsDir)
cliflagcfg.StringFlag(f, &testDirectorySvrContext.tenantBaseDir, cliflags.TestDirectoryTenantBaseDir)
// Use StringFlagDepth to avoid conflicting with the already registered KVAddrs env var.
cliflagcfg.StringFlagDepth(1, f, &testDirectorySvrContext.kvAddrs, cliflags.KVAddrs)
})
}
18 changes: 18 additions & 0 deletions pkg/ccl/cliccl/mt.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
// Copyright 2022 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package cliccl

import "github.com/cockroachdb/cockroach/pkg/cli"

func init() {
cli.MTCmd.AddCommand(mtStartSQLProxyCmd)
cli.RegisterCommandWithCustomLogging(mtStartSQLProxyCmd)
cli.MTCmd.AddCommand(mtTestDirectorySvr)
cli.RegisterCommandWithCustomLogging(mtTestDirectorySvr)
}
29 changes: 10 additions & 19 deletions pkg/cli/mt_proxy.go → pkg/ccl/cliccl/mt_proxy.go
Original file line number Diff line number Diff line change
@@ -1,14 +1,12 @@
// Copyright 2021 The Cockroach Authors.
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// 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.
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package cli
package cliccl

import (
"context"
Expand All @@ -19,6 +17,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl"
"github.com/cockroachdb/cockroach/pkg/cli"
"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/severity"
Expand Down Expand Up @@ -97,18 +96,10 @@ func runStartSQLProxy(cmd *cobra.Command, args []string) (returnErr error) {
}

func initLogging(cmd *cobra.Command) (ctx context.Context, stopper *stop.Stopper, err error) {
// Remove the default store, which avoids using it to set up logging.
// Instead, we'll default to logging to stderr unless --log-dir is
// specified. This makes sense since the standalone SQL server is
// at the time of writing stateless and may not be provisioned with
// suitable storage.
serverCfg.Stores.Specs = nil
serverCfg.ClusterName = ""

ctx = context.Background()
stopper, err = setupAndInitializeLoggingAndProfiling(ctx, cmd, false /* isServerCmd */)
stopper, err = cli.ClearStoresAndSetupLoggingForMTCommands(cmd, ctx)
if err != nil {
return
return ctx, nil, err
}
ctx, _ = stopper.WithCancelOnQuiesce(ctx)
return ctx, stopper, err
Expand All @@ -123,7 +114,7 @@ func waitForSignals(
) (returnErr error) {
// Need to alias the signals if this has to run on non-unix OSes too.
signalCh := make(chan os.Signal, 1)
signal.Notify(signalCh, drainSignals...)
signal.Notify(signalCh, cli.DrainSignals...)

select {
case err := <-errChan:
Expand Down
Loading

0 comments on commit a30fb14

Please sign in to comment.