Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…77019 #77045 #77047 #77049

72925: sql, cli: support basic auto complete for sql keywords r=rafiss a=RichardJCai

sql: add SHOW COMPLETIONS AT offset FOR syntax 

Release note (sql change): Support
SHOW COMPLETIONS AT OFFSET <offset> FOR <stmt> syntax that
returns a set of SQL keywords that can complete the keyword at
<offset> in the given <stmt>.

If the offset is in the middle of a word, then it returns the
full word.
For example SHOW COMPLETIONS AT OFFSET 1 FOR "SELECT" returns select.

cli: support autocomplete 

Release note (cli change): CLI now auto completes on tab
by using `SHOW COMPLETIONS AT OFFSET`.

76539: cli: Enable profiles and other debug info for tenants r=rimadeodhar a=rimadeodhar

This PR updates debug.zip functionality to
collect goroutine stacks and profiles for all
active SQL instances for a tenant.
This PR also addresses a bug where the nodes.json
and status.json data was not getting populated
correctly due to the switch to the `NodesList` API.
This bug has been addressed by using the p;d
`Nodes` API when the debug zip command is run against
a storage server.

Release note: None

76676: telemetry,sql: remove redaction from operational sql data r=abarganier a=dhartunian

Previously, when redaction was introduced into CRDB, all unidentifiable
strings were marked as redacted since that was the safer approach. We
expected to later return with a closer look and differentiate more
carefully between what should be redacted and what shouldn't.

SQL names have been identified as operational-sensitive data that should
not be redacted since it provides very useful debugging information and,
while user-controlled, do not typically contain user-data since those
would be stored in a Datum. This commit marks names as safe from
redaction for telemetry logging in cases where the
`sql.telemetry.query_sampling.enabled` cluster setting is enabled.

Additionally, some log tags such as client IP addresses are not to be
considered sensitive and are critical to debugging operational issues.
They have also been marked as safe.

In order to help with documenting these cases, a helper
`SafeOperational()` has been added to the `log` package. This helps us
mark strings as safe while documenting *why* we're doing so.

Resolves #76595

Release note (security update, ops change): When the
`sql.telemetry.query_sampling.enabled` cluster setting is enabled, SQL
names and client IPs are no longer redacted in telemetry logs.

76754: physicalplan: add support for multi-stage execution of corr, covar_samp, sqrdiff, and regr_count aggregate functions. r=yuzefovich a=mneverov

Fixes: #58347.

Release note (performance improvement): corr, covar_samp, sqrdiff, and
regr_count aggregate functions are now evaluated more efficiently in a
distributed setting

76908: roachtest: update 22.1 version map to v21.2.6 r=bananabrick a=bananabrick

Release note: None

76948: spanconfigreconciler{ccl}: apply system span config diffs to the store r=arulajmani a=adityamaru

This change teaches the reconciler about system span configs. Concretely,
we make the following changes:

- A full reconciliation when checking for existing span configurations now
asks for SpanConfigs corresponding to the SystemTargets relevant to the tenant.

For the host tenant this includes the SystemTarget for the `entire-keyspace` as
well as the SystemTarget for span configs installed by the host tenant on its
tenant keyspace, and on other secondary tenant keyspaces.

For secondary tenants this only includes the SystemTarget for span configs installed
by it on its own tenant keyspace.

- During incremental reconciliation, before applying our updates to the Store,
we now also check for "missing protected timestamp system targets". These correspond
to protected timestamp records that target a `Cluster` or a `Tenant` but no longer
exist in the system.protected_ts_records table as they have been released by the client.
For every such unique missing system target we apply a spanconfig.Deletion to the Store.

In order to make the above possible, this change moves the ptsStateReader from the
`spanconfigsqltranslator` package, to the top level `spanconfig` package.

Informs: #73727

Release note: None

76990: sql, geo: Fix upper case geohash parsing and allow NULL arguments r=otan a=RichardJCai

Release note (sql change): ST_Box2DFromGeoHash now accepts NULL arguments,
the precision being NULL is the same as no precision being passed in at all.

Upper case characters are now parsed as lower case characters for geohash,
this matches Postgis behaviour.

Resolves #75537

77006: changefeedccl: Fix data race in a test. r=miretskiy a=miretskiy

Fix data race in TestChangefeedSendError.

Release Notes: None

77014: log: Set content type header for http sink r=dhartunian a=rimadeodhar

The content type header for the output of HTTP log sink
is always set to text/plain irrespective of the log format.
If the log format is JSON, we should set the content
type to be application/json.

Release note (bug fix): The content type header for the
HTTP log sink is set to application/json if the format of
the log output is JSON.

77019: sql: update tpcc and tpch stats r=rharding6373 a=rharding6373

This change rewrites the stats for tpcc and tpch to include the new
table statistic avg_size.

Fixes: #72332

Release note: None

77045: bazel: bump `rules_go` to pick up cockroachdb/rules_go#4 r=irfansharif a=rickystewart

Closes #77037.

Release note: None

77047: dev: make sure we inherit `stdout` and `stderr` when appropriate r=postamar,irfansharif a=rickystewart

Otherwise `bazel build`/`test` output will be ugly (won't be colored/
use `ncurses`/etc.)

Release note: None

77049: sql: skip flaky schema_changer/drop_database_cascade test r=postamar a=RichardJCai

Release note: None

Co-authored-by: richardjcai <[email protected]>
Co-authored-by: rimadeodhar <[email protected]>
Co-authored-by: David Hartunian <[email protected]>
Co-authored-by: Max Neverov <[email protected]>
Co-authored-by: Arjun Nair <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: rharding6373 <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
10 people committed Feb 25, 2022
14 parents c3ff362 + 1e64f18 + fd6c580 + 10add46 + 6fb88ef + 582e650 + c300ec2 + 537f89d + 914f3b6 + 65c59a6 + 0048936 + dcbfa39 + 11e09c3 + dbe8bef commit 2826900
Show file tree
Hide file tree
Showing 112 changed files with 35,436 additions and 18,697 deletions.
8 changes: 4 additions & 4 deletions WORKSPACE
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,12 @@ load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive")
# Load go bazel tools. This gives us access to the go bazel SDK/toolchains.
http_archive(
name = "io_bazel_rules_go",
sha256 = "5c4bd27429b1a307d51cd23d4677126aa6315fff608f0cd85c5bfb642a13b953",
strip_prefix = "cockroachdb-rules_go-23b381c",
sha256 = "a80c384a85ab9f6318419ed38a39b13e8481e9d2ea25a696ea9e25438e017b5f",
strip_prefix = "cockroachdb-rules_go-f96cc04",
urls = [
# cockroachdb/rules_go as of 23b381cc8a679f5b10fd2459fc87997728b3740c
# cockroachdb/rules_go as of f96cc0445102a7b08bceeb0bba32b4d82467a065
# (upstream release-0.29 plus a few patches).
"https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-52-g23b381c.tar.gz",
"https://storage.googleapis.com/public-bazel-artifacts/bazel/cockroachdb-rules_go-v0.27.0-54-gf96cc04.tar.gz",
],
)

Expand Down
2 changes: 1 addition & 1 deletion dev
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
set -euo pipefail

# Bump this counter to force rebuilding `dev` on all machines.
DEV_VERSION=18
DEV_VERSION=19

THIS_DIR=$(cd "$(dirname "$0")" && pwd)
BINARY_DIR=$THIS_DIR/bin/dev-versions
Expand Down
6 changes: 3 additions & 3 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -2705,7 +2705,7 @@ Support status: [reserved](#support-status)

| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.StacksRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. | [reserved](#support-status) |
| node_id | [string](#cockroach.server.serverpb.StacksRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. node_id translates to a KV node ID on a storage server and SQL instance ID on a SQL only server. | [reserved](#support-status) |
| type | [StacksType](#cockroach.server.serverpb.StacksRequest-cockroach.server.serverpb.StacksType) | | | [reserved](#support-status) |


Expand Down Expand Up @@ -2750,7 +2750,7 @@ Support status: [reserved](#support-status)

| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.ProfileRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. | [reserved](#support-status) |
| node_id | [string](#cockroach.server.serverpb.ProfileRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. node_id translates to a KV node ID on a storage server and SQL instance ID on a SQL only server. | [reserved](#support-status) |
| type | [ProfileRequest.Type](#cockroach.server.serverpb.ProfileRequest-cockroach.server.serverpb.ProfileRequest.Type) | | The type of profile to retrieve. | [reserved](#support-status) |
| seconds | [int32](#cockroach.server.serverpb.ProfileRequest-int32) | | applies only to Type=CPU, defaults to 30 | [reserved](#support-status) |
| labels | [bool](#cockroach.server.serverpb.ProfileRequest-bool) | | applies only to Type=CPU, defaults to false | [reserved](#support-status) |
Expand Down Expand Up @@ -2845,7 +2845,7 @@ Support status: [reserved](#support-status)

| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.GetFilesRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. | [reserved](#support-status) |
| node_id | [string](#cockroach.server.serverpb.GetFilesRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. node_id translates to a KV node ID on a storage server and SQL instance ID on a SQL only server. | [reserved](#support-status) |
| list_only | [bool](#cockroach.server.serverpb.GetFilesRequest-bool) | | If list_only is true then the contents of the files will not be populated in the response. Only filenames and sizes will be returned. | [reserved](#support-status) |
| type | [FileType](#cockroach.server.serverpb.GetFilesRequest-cockroach.server.serverpb.FileType) | | | [reserved](#support-status) |
| patterns | [string](#cockroach.server.serverpb.GetFilesRequest-string) | repeated | Each pattern given is matched with Files of the above type in the node using filepath.Glob(). The patterns only match to filenames and so path separators cannot be used. Example: * will match all files of requested type. | [reserved](#support-status) |
Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -934,6 +934,7 @@ unreserved_keyword ::=
| 'COMMITTED'
| 'COMPACT'
| 'COMPLETE'
| 'COMPLETIONS'
| 'CONFLICT'
| 'CONFIGURATION'
| 'CONFIGURATIONS'
Expand Down
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -255,6 +255,7 @@ ALL_TESTS = [
"//pkg/sql/contention/txnidcache:txnidcache_test",
"//pkg/sql/contention:contention_test",
"//pkg/sql/covering:covering_test",
"//pkg/sql/delegate:delegate_test",
"//pkg/sql/distsql:distsql_test",
"//pkg/sql/doctor:doctor_test",
"//pkg/sql/enum:enum_test",
Expand Down
12 changes: 12 additions & 0 deletions pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,4 +312,16 @@ type TestTenantArgs struct {

// RPCHeartbeatInterval controls how often the tenant sends Ping requests.
RPCHeartbeatInterval time.Duration

// GoroutineDumpDirName is used to initialize the same named field on the
// SQLServer.BaseConfig field. It is used as the directory name for
// goroutine dumps using goroutinedumper. If set, this directory should
// be cleaned up once the test completes.
GoroutineDumpDirName string

// HeapProfileDirName is used to initialize the same named field on the
// SQLServer.BaseConfig field. It is the directory name for heap profiles using
// heapprofiler. If empty, no heap profiles will be collected during the test.
// If set, this directory should be cleaned up after the test completes.
HeapProfileDirName string
}
13 changes: 7 additions & 6 deletions pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,21 +212,22 @@ func TestChangefeedSendError(t *testing.T) {
Changefeed.(*TestingKnobs)

// Allow triggering a single sendError
var sendError int32 = 0
sendErrorCh := make(chan error, 1)
knobs.FeedKnobs.OnRangeFeedValue = func(_ roachpb.KeyValue) error {
if sendError != 0 {
atomic.StoreInt32(&sendError, 0)
return kvcoord.TestNewSendError("test sendError")
select {
case err := <-sendErrorCh:
return err
default:
return nil
}
return nil
}

foo := feed(t, f, `CREATE CHANGEFEED FOR foo`)
defer closeFeed(t, foo)

sqlDB.Exec(t, `INSERT INTO foo VALUES (1)`)
sqlDB.Exec(t, `INSERT INTO foo VALUES (2)`)
atomic.StoreInt32(&sendError, 1)
sendErrorCh <- kvcoord.TestNewSendError("test sendError")
sqlDB.Exec(t, `INSERT INTO foo VALUES (3)`)
sqlDB.Exec(t, `INSERT INTO foo VALUES (4)`)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ func TestDataDriven(t *testing.T) {
// configuration changes
testutils.SucceedsSoon(t, func() error {
for _, tenant := range spanConfigTestCluster.Tenants() {
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastExec()
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastSQLChange()
if lastCheckpoint.IsEmpty() {
continue // reconciler wasn't started
}
Expand Down Expand Up @@ -191,7 +191,7 @@ func TestDataDriven(t *testing.T) {
case "exec-sql":
// Run under an explicit transaction -- we rely on having a
// single timestamp for the statements (see
// tenant.TimestampAfterLastExec) for ordering guarantees.
// tenant.TimestampAfterLastSQLChange) for ordering guarantees.
tenant.Exec(fmt.Sprintf("BEGIN; %s; COMMIT;", d.Input))

case "query-sql":
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/spanconfigccl/spanconfigreconcilerccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_test(
"//pkg/ccl/partitionccl",
"//pkg/ccl/utilccl",
"//pkg/jobs",
"//pkg/kv/kvserver/protectedts",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/securitytest",
Expand Down
45 changes: 39 additions & 6 deletions pkg/ccl/spanconfigccl/spanconfigreconcilerccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl"
_ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils"
Expand Down Expand Up @@ -60,6 +61,16 @@ import (
// Print out the contents of KVAccessor directly, skipping 'offset' entries,
// returning up to the specified limit if any.
//
// - "protect" [record-id=<int>] [ts=<int>]
// cluster OR
// tenants id1,id2... OR
// descs id1,id2...
// Creates and writes a protected timestamp record with id and ts with an
// appropriate ptpb.Target.
//
// - "release" [record-id=<int>]
// Releases the protected timestamp record with id.
//
// TODO(irfansharif): Provide a way to stop reconcilers and/or start them back
// up again. It would let us add simulate for suspended tenants, and behavior of
// the reconciler with existing kvaccessor state (populated by an earlier
Expand All @@ -85,11 +96,13 @@ func TestDataDriven(t *testing.T) {
// Checkpoint noops frequently; speeds this test up.
SQLWatcherCheckpointNoopsEveryDurationOverride: 100 * time.Millisecond,
}
ptsKnobs := &protectedts.TestingKnobs{EnableProtectedTimestampForMultiTenant: true}
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test
SpanConfig: scKnobs,
ProtectedTS: ptsKnobs,
},
},
})
Expand All @@ -101,7 +114,7 @@ func TestDataDriven(t *testing.T) {
tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'`)
}

spanConfigTestCluster := spanconfigtestcluster.NewHandle(t, tc, scKnobs, nil /* ptsKnobs */)
spanConfigTestCluster := spanconfigtestcluster.NewHandle(t, tc, scKnobs, ptsKnobs)
defer spanConfigTestCluster.Cleanup()

systemTenant := spanConfigTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID)
Expand Down Expand Up @@ -131,7 +144,7 @@ func TestDataDriven(t *testing.T) {
case "exec-sql":
// Run under an explicit transaction -- we rely on having a
// single timestamp for the statements (see
// tenant.TimestampAfterLastExec) for ordering guarantees.
// tenant.TimestampAfterLastSQLChange) for ordering guarantees.
tenant.Exec(fmt.Sprintf("BEGIN; %s; COMMIT;", d.Input))

case "query-sql":
Expand Down Expand Up @@ -159,7 +172,7 @@ func TestDataDriven(t *testing.T) {

case "mutations":
testutils.SucceedsSoon(t, func() error {
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastExec()
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastSQLChange()
if lastCheckpoint.Less(lastExec) {
return errors.Newf("last checkpoint timestamp (%s) lagging last sql execution (%s)",
lastCheckpoint.GoTime(), lastExec.GoTime())
Expand All @@ -179,7 +192,7 @@ func TestDataDriven(t *testing.T) {
// tenant checkpoints to cross their last execution
// timestamp.
for _, tenant := range spanConfigTestCluster.Tenants() {
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastExec()
lastCheckpoint, lastExec := tenant.LastCheckpoint(), tenant.TimestampAfterLastSQLChange()
if lastCheckpoint.IsEmpty() {
continue // reconciler wasn't started
}
Expand All @@ -200,11 +213,31 @@ func TestDataDriven(t *testing.T) {

lines := make([]string, len(records))
for i, record := range records {
lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSpan().String(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config))
switch {
case record.Target.IsSpanTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSpan(),
spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(record.Config))
case record.Target.IsSystemTarget():
lines[i] = fmt.Sprintf("%-42s %s", record.Target.GetSystemTarget(),
spanconfigtestutils.PrintSystemSpanConfigDiffedAgainstDefault(record.Config))
default:
panic("unsupported target type")
}
}
return spanconfigtestutils.MaybeLimitAndOffset(t, d, "...", lines)

case "protect":
var recordID string
var protectTS int
d.ScanArgs(t, "record-id", &recordID)
d.ScanArgs(t, "ts", &protectTS)
target := spanconfigtestutils.ParseProtectionTarget(t, d.Input)
tenant.MakeProtectedTimestampRecordAndProtect(ctx, recordID, protectTS, target)

case "release":
var recordID string
d.ScanArgs(t, "record-id", &recordID)
tenant.ReleaseProtectedTimestampRecord(ctx, recordID)
default:
t.Fatalf("unknown command: %s", d.Cmd)
}
Expand Down
Loading

0 comments on commit 2826900

Please sign in to comment.