diff --git a/build/bazelutil/nogo_config.json b/build/bazelutil/nogo_config.json index be0c8326323a..df1ec3c0b715 100644 --- a/build/bazelutil/nogo_config.json +++ b/build/bazelutil/nogo_config.json @@ -53,7 +53,8 @@ "cockroach/pkg/sql/pgwire/pgerror/pgcode\\.go$": "invalid direct cast on error object", "cockroach/pkg/testutils/lint/lint_test\\.go$": "invalid direct cast on error object", "cockroach/pkg/util/contextutil/timeout_error\\.go$": "invalid direct cast on error object", - "cockroach/pkg/util/sysutil/sysutil_.*": "type can change by system" + "cockroach/pkg/util/sysutil/sysutil_.*": "type can change by system", + "cockroach/pkg/cloud/gcp/gcs_retry\\.go$": "invalid direct cast on error object" }, "only_files": { "cockroach/pkg/.*$": "first-party code" diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 1cb987f34c95..17cfa5fa35a0 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2463,7 +2463,6 @@ contains common SQL event/execution details. | `Database` | Name of the database that initiated the query. | no | | `StatementID` | Statement ID of the query. | no | | `TransactionID` | Transaction ID of the query. | no | -| `DatabaseID` | Database ID of the query. | no | | `StatementFingerprintID` | Statement fingerprint ID of the query. | no | | `MaxFullScanRowsEstimate` | Maximum number of rows scanned by a full scan, as estimated by the optimizer. | no | | `TotalScanRowsEstimate` | Total number of rows read by all scans in the query, as estimated by the optimizer. | no | diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 2a1c413dfd9a..9e9b4a3477d5 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -2160,7 +2160,10 @@ array_expr ::= backup_options ::= 'ENCRYPTION_PASSPHRASE' '=' string_or_placeholder | 'REVISION_HISTORY' + | 'REVISION_HISTORY' '=' a_expr | 'DETACHED' + | 'DETACHED' '=' 'TRUE' + | 'DETACHED' '=' 'FALSE' | 'KMS' '=' string_or_placeholder_opt_list | 'INCREMENTAL_LOCATION' '=' string_or_placeholder_opt_list diff --git a/docs/tech-notes/mvcc-range-tombstones.md b/docs/tech-notes/mvcc-range-tombstones.md index d39104eeb287..e16ad702c31b 100644 --- a/docs/tech-notes/mvcc-range-tombstones.md +++ b/docs/tech-notes/mvcc-range-tombstones.md @@ -75,8 +75,8 @@ type MVCCRangeKey struct { } ``` -A range key stores an encoded `MVCCValue`, similarly to `MVCCKey`. They are -often paired as an `MVCCRangeKeyValue`: +A range key stores an encoded `MVCCValue`, similarly to `MVCCKey`. They can be +paired as an `MVCCRangeKeyValue`: ```go type MVCCRangeKeyValue struct { @@ -122,12 +122,31 @@ exist between the bounds `[a-d)`, and `c` is within those bounds. The same is true for `a@5`, even though it is above both MVCC range tombstones. It is up to the iterator caller to interpret the range keys as appropriate relative to the point key. It follows that all range keys overlapping a key will be pulled into -memory at once, but we assume that overlapping range keys will be few. More on -MVCC iteration later. +memory at once, but we assume that overlapping range keys will be few. -In the KV API, however, this distinction doesn't really matter: `Get(c)` at -timestamp >= 5 would return nothing, while `Get(b)` would return `b5`. Again, -more on this later. +This is represented as a specialized compact data structure, +`MVCCRangeKeyStack`, where all range keys have the same bounds due to +fragmentation (described below): + +```go +type MVCCRangeKeyStack struct { + Bounds roachpb.Span + Versions MVCCRangeKeyVersions +} + +type MVCCRangeKeyVersions []MVCCRangeKeyVersion + +type MVCCRangeKeyVersion struct { + Timestamp hlc.Timestamp + Value []byte // encoded MVCCValue +} +``` + +In the KV API, however, the relationship between point keys and range keys +doesn't really matter: `Get(c)` at timestamp >= 5 would simply return nothing, +while `Get(b)` would return `b5`. More on this later. + +### Fragmentation Range keys do not have a stable, discrete identity, and should be considered a continuum: they may be partially removed or replaced, merged or fragmented by @@ -161,7 +180,7 @@ Pebble: `[a-b)@1`, `[b-c)@2`, `[b-c)@1`, and `[c-d)@2`. Similarly, clearing `[b-d)@2` would merge the remaining keys back into `[a-c)@1`. This implies that all range keys exposed for a specific key position all have -the same key bounds. +the same key bounds, as shown in `MVCCRangeKeyStack`. Fragmentation is beneficial because it makes all range key properties local, which avoids incurring unnecessary access costs across SSTs and CRDB ranges when @@ -268,7 +287,7 @@ The properties of point and range keys are accessed via: * `RangeBounds()`: start and end bounds of range keys overlapping the current position, if any. * `RangeKeys()`: all range keys at the current key position (i.e. at all - timestamps), as `[]MVCCRangeKeyValue`. + timestamps), as `MVCCRangeKeyStack`. During iteration with `IterKeyTypePointsAndRanges`, range keys are emitted at their start key and at every overlapping point key. Consider a modified diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index ee7e5dab9542..4f58ffd7afda 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -426,6 +426,18 @@ func backupPlanHook( return nil, nil, nil, false, err } + detached := false + if backupStmt.Options.Detached == tree.DBoolTrue { + detached = true + } + revisionHistoryFn := func() (bool, error) { return false, nil } // Defaults to false. + if backupStmt.Options.CaptureRevisionHistory != nil { + revisionHistoryFn, err = p.TypeAsBool(ctx, backupStmt.Options.CaptureRevisionHistory, "BACKUP") + if err != nil { + return nil, nil, nil, false, err + } + } + encryptionParams := jobspb.BackupEncryptionOptions{Mode: jobspb.EncryptionMode_None} var pwFn func() (string, error) @@ -464,7 +476,7 @@ func backupPlanHook( ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag()) defer span.Finish() - if !(p.ExtendedEvalContext().TxnIsSingleStmt || backupStmt.Options.Detached) { + if !(p.ExtendedEvalContext().TxnIsSingleStmt || detached) { return errors.Errorf("BACKUP cannot be used inside a multi-statement transaction without DETACHED option") } @@ -529,12 +541,14 @@ func backupPlanHook( } } - var revisionHistory bool - if backupStmt.Options.CaptureRevisionHistory { + revisionHistory, err := revisionHistoryFn() + if err != nil { + return err + } + if revisionHistory { if err := requireEnterprise(p.ExecCfg(), "revision_history"); err != nil { return err } - revisionHistory = true } var targetDescs []catalog.Descriptor @@ -633,7 +647,7 @@ func backupPlanHook( } plannerTxn := p.Txn() - if backupStmt.Options.Detached { + if detached { // When running inside an explicit transaction, we simply create the job // record. We do not wait for the job to finish. _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( @@ -674,7 +688,7 @@ func backupPlanHook( return sj.ReportExecutionResults(ctx, resultsCh) } - if backupStmt.Options.Detached { + if detached { return fn, jobs.DetachedJobExecutionResultHeader, nil, false, nil } return fn, jobs.BulkJobExecutionResultHeader, nil, false, nil diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index a11cd08f3bda..3d5f1169da09 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -5583,7 +5583,7 @@ func TestBackupRestoreShowJob(t *testing.T) { sqlDB.CheckQueryResults( t, "SELECT description FROM [SHOW JOBS] WHERE description != 'updating privileges' ORDER BY description", [][]string{ - {"BACKUP DATABASE data TO 'nodelocal://0/foo' WITH revision_history"}, + {"BACKUP DATABASE data TO 'nodelocal://0/foo' WITH revision_history = true"}, {"RESTORE TABLE data.bank FROM 'nodelocal://0/foo' WITH into_db = 'data 2', skip_missing_foreign_keys"}, }, ) diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 7b57b2a71cbc..08d1ddfaf89c 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -296,7 +296,7 @@ func doCreateBackupSchedules( backupNode := &tree.Backup{ Options: tree.BackupOptions{ CaptureRevisionHistory: eval.BackupOptions.CaptureRevisionHistory, - Detached: true, + Detached: tree.DBoolTrue, }, Nested: true, AppendToLatest: false, diff --git a/pkg/ccl/backupccl/create_scheduled_backup_test.go b/pkg/ccl/backupccl/create_scheduled_backup_test.go index 2309cefdb413..df41931b2e12 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup_test.go +++ b/pkg/ccl/backupccl/create_scheduled_backup_test.go @@ -422,7 +422,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "BACKUP .+", - backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history = true, detached", period: time.Hour, }, }, @@ -459,7 +459,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { nameRe: "BACKUP .*", backupStmt: "BACKUP TABLE system.public.jobs, " + "system.public.scheduled_jobs INTO LATEST IN 'nodelocal://0/backup' WITH" + - " revision_history, detached", + " revision_history = true, detached", period: time.Hour, paused: true, chainProtectedTimestampRecord: true, @@ -467,7 +467,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { { nameRe: "BACKUP .+", backupStmt: "BACKUP TABLE system.public.jobs, " + - "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history, detached", + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH revision_history = true, detached", period: 24 * time.Hour, runsNow: true, chainProtectedTimestampRecord: true, @@ -483,14 +483,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "BACKUP .*", - backupStmt: "BACKUP DATABASE system INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP DATABASE system INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached", period: time.Hour, paused: true, chainProtectedTimestampRecord: true, }, { nameRe: "BACKUP .+", - backupStmt: "BACKUP DATABASE system INTO 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP DATABASE system INTO 'nodelocal://0/backup' WITH revision_history = true, detached", period: 24 * time.Hour, runsNow: true, chainProtectedTimestampRecord: true, @@ -506,14 +506,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "BACKUP .*", - backupStmt: "BACKUP TABLE system.public.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP TABLE system.public.* INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached", period: time.Hour, paused: true, chainProtectedTimestampRecord: true, }, { nameRe: "BACKUP .+", - backupStmt: "BACKUP TABLE system.public.* INTO 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP TABLE system.public.* INTO 'nodelocal://0/backup' WITH revision_history = true, detached", period: 24 * time.Hour, runsNow: true, chainProtectedTimestampRecord: true, @@ -546,14 +546,14 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { expectedSchedules: []expectedSchedule{ { nameRe: "my_backup_name", - backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH revision_history = true, detached", period: time.Hour, paused: true, chainProtectedTimestampRecord: true, }, { nameRe: "my_backup_name", - backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached", + backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history = true, detached", period: 24 * time.Hour, runsNow: true, chainProtectedTimestampRecord: true, @@ -571,10 +571,10 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { nameRe: "BACKUP .*", backupStmt: "BACKUP TABLE system.public.jobs, " + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" + - " revision_history, encryption_passphrase = 'secret', detached", + " revision_history = true, encryption_passphrase = 'secret', detached", shownStmt: "BACKUP TABLE system.public.jobs, " + "system.public.scheduled_jobs INTO 'nodelocal://0/backup' WITH" + - " revision_history, encryption_passphrase = '*****', detached", + " revision_history = true, encryption_passphrase = '*****', detached", period: 7 * 24 * time.Hour, }, }, @@ -596,7 +596,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { nameRe: "BACKUP .+", backupStmt: "BACKUP DATABASE system INTO " + "('nodelocal://0/backup?COCKROACH_LOCALITY=x%3Dy', 'nodelocal://0/backup2?COCKROACH_LOCALITY=default') " + - "WITH revision_history, detached", + "WITH revision_history = true, detached", period: 24 * time.Hour, }, }, @@ -614,7 +614,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { }, } - for _, tc := range testCases { + for i, tc := range testCases { t.Run(fmt.Sprintf("%s-%s", tc.name, tc.user), func(t *testing.T) { defer th.clearSchedules(t) @@ -652,7 +652,7 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) { for _, s := range schedules { stmt := getScheduledBackupStatement(t, s.ExecutionArgs()) expectedSchedule, ok := expectedByName[stmt] - require.True(t, ok, "could not find matching name for %q", stmt) + require.True(t, ok, "in test case %d, could not find matching name for %q", i, stmt) require.Regexp(t, regexp.MustCompile(expectedSchedule.nameRe), s.ScheduleLabel()) expectedShown := fmt.Sprintf("%q", expectedSchedule.backupStmt) diff --git a/pkg/ccl/backupccl/file_sst_sink.go b/pkg/ccl/backupccl/file_sst_sink.go index b5c49da6d13e..7dffa08e944b 100644 --- a/pkg/ccl/backupccl/file_sst_sink.go +++ b/pkg/ccl/backupccl/file_sst_sink.go @@ -363,8 +363,9 @@ func (s *fileSSTSink) copyRangeKeys(dataSST []byte) error { } else if !ok { break } - for _, rkv := range iter.RangeKeys() { - if err := s.sst.PutRawMVCCRangeKey(rkv.RangeKey, rkv.Value); err != nil { + rangeKeys := iter.RangeKeys() + for _, v := range rangeKeys.Versions { + if err := s.sst.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), v.Value); err != nil { return err } } diff --git a/pkg/ccl/backupccl/schedule_exec.go b/pkg/ccl/backupccl/schedule_exec.go index 8f975220a65a..62f749659102 100644 --- a/pkg/ccl/backupccl/schedule_exec.go +++ b/pkg/ccl/backupccl/schedule_exec.go @@ -70,8 +70,8 @@ func (e *scheduledBackupExecutor) executeBackup( } // Sanity check: backup should be detached. - if !backupStmt.Options.Detached { - backupStmt.Options.Detached = true + if backupStmt.Options.Detached != tree.DBoolTrue { + backupStmt.Options.Detached = tree.DBoolTrue log.Warningf(ctx, "force setting detached option for backup schedule %d", sj.ScheduleID()) } diff --git a/pkg/cloud/gcp/BUILD.bazel b/pkg/cloud/gcp/BUILD.bazel index 81407c889602..db08e83e80d6 100644 --- a/pkg/cloud/gcp/BUILD.bazel +++ b/pkg/cloud/gcp/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "gcp", srcs = [ "gcs_kms.go", + "gcs_retry.go", "gcs_storage.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/cloud/gcp", @@ -23,11 +24,15 @@ go_library( "@com_github_gogo_protobuf//types", "@com_google_cloud_go_kms//apiv1", "@com_google_cloud_go_storage//:storage", + "@org_golang_google_api//googleapi", "@org_golang_google_api//impersonate", "@org_golang_google_api//iterator", "@org_golang_google_api//option", "@org_golang_google_genproto//googleapis/cloud/kms/v1:kms", + "@org_golang_google_grpc//codes", + "@org_golang_google_grpc//status", "@org_golang_google_protobuf//types/known/wrapperspb", + "@org_golang_x_net//http2", "@org_golang_x_oauth2//:oauth2", ], ) diff --git a/pkg/cloud/gcp/gcs_retry.go b/pkg/cloud/gcp/gcs_retry.go new file mode 100644 index 000000000000..f70051c18ff9 --- /dev/null +++ b/pkg/cloud/gcp/gcs_retry.go @@ -0,0 +1,78 @@ +// Copyright 2022 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 gcp + +import ( + "io" + "net" + "net/url" + "strings" + + "github.com/cockroachdb/errors" + "google.golang.org/api/googleapi" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) + +// defaultShouldRetry is google-cloud's default predicate for determining +// whether an error can be retried. +// +// TODO(rui): Currently this code is copied as-is from the google-cloud-go SDK +// in order to get the default retry behavior on top of our own customizations. +// There's currently a PR in google-cloud-go that exposes the default retry +// function, so this can be removed when it is merged: +// https://github.com/googleapis/google-cloud-go/pull/6370 +func defaultShouldRetry(err error) bool { + if err == nil { + return false + } + if errors.Is(err, io.ErrUnexpectedEOF) { + return true + } + + switch e := err.(type) { + case *net.OpError: + if strings.Contains(e.Error(), "use of closed network connection") { + // TODO: check against net.ErrClosed (go 1.16+) instead of string + return true + } + case *googleapi.Error: + // Retry on 408, 429, and 5xx, according to + // https://cloud.google.com/storage/docs/exponential-backoff. + return e.Code == 408 || e.Code == 429 || (e.Code >= 500 && e.Code < 600) + case *url.Error: + // Retry socket-level errors ECONNREFUSED and ECONNRESET (from syscall). + // Unfortunately the error type is unexported, so we resort to string + // matching. + retriable := []string{"connection refused", "connection reset"} + for _, s := range retriable { + if strings.Contains(e.Error(), s) { + return true + } + } + case interface{ Temporary() bool }: + if e.Temporary() { + return true + } + } + // HTTP 429, 502, 503, and 504 all map to gRPC UNAVAILABLE per + // https://grpc.github.io/grpc/core/md_doc_http-grpc-status-mapping.html. + // + // This is only necessary for the experimental gRPC-based media operations. + if st, ok := status.FromError(err); ok && st.Code() == codes.Unavailable { + return true + } + // Unwrap is only supported in go1.13.x+ + if e, ok := err.(interface{ Unwrap() error }); ok { + return defaultShouldRetry(e.Unwrap()) + } + return false +} diff --git a/pkg/cloud/gcp/gcs_storage.go b/pkg/cloud/gcp/gcs_storage.go index 9787007b2c01..8cc9fa31946d 100644 --- a/pkg/cloud/gcp/gcs_storage.go +++ b/pkg/cloud/gcp/gcs_storage.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/gogo/protobuf/types" + "golang.org/x/net/http2" "golang.org/x/oauth2" "google.golang.org/api/impersonate" "google.golang.org/api/iterator" @@ -174,6 +175,7 @@ func makeGCSStorage( if err != nil { return nil, errors.Wrap(err, "failed to create google cloud client") } + g.SetRetry(gcs.WithErrorFunc(shouldRetry)) bucket := g.Bucket(conf.Bucket) if conf.BillingProject != `` { bucket = bucket.UserProject(conf.BillingProject) @@ -343,6 +345,34 @@ func (g *gcsStorage) Close() error { return g.client.Close() } +// shouldRetry is the predicate that determines whether a GCS client error +// should be retried. The predicate combines google-cloud-go's default retry +// predicate and some additional predicates when determining whether the error +// is retried. The additional predicates are: +// +// - http2.StreamError error with code http2.ErrCodeInternal: this error has +// been recommended to be retried in several issues in the google-cloud-go repo: +// https://github.com/googleapis/google-cloud-go/issues/3735 +// https://github.com/googleapis/google-cloud-go/issues/784 +// Remove if this error ever becomes part of the default retry predicate. +func shouldRetry(err error) bool { + if defaultShouldRetry(err) { + return true + } + + if e := (http2.StreamError{}); errors.As(err, &e) { + if e.Code == http2.ErrCodeInternal { + return true + } + } + + if e := (errors.Wrapper)(nil); errors.As(err, &e) { + return shouldRetry(e.Unwrap()) + } + + return false +} + func init() { cloud.RegisterExternalStorageProvider(cloudpb.ExternalStorageProvider_gs, parseGSURL, makeGCSStorage, cloud.RedactedParams(CredentialsParam, BearerTokenParam), "gs") diff --git a/pkg/cmd/dev/test.go b/pkg/cmd/dev/test.go index 721b5dbf3750..7569213a40fc 100644 --- a/pkg/cmd/dev/test.go +++ b/pkg/cmd/dev/test.go @@ -135,6 +135,7 @@ func (d *dev) test(cmd *cobra.Command, commandLine []string) error { extraRewritablePaths = []struct{ pkg, path string }{ {"pkg/ccl/logictestccl", "pkg/sql/logictest"}, {"pkg/sql/opt/memo", "pkg/sql/opt/testutils/opttester/testfixtures"}, + {"pkg/sql/opt/norm", "pkg/sql/opt/testutils/opttester/testfixtures"}, {"pkg/sql/opt/xform", "pkg/sql/opt/testutils/opttester/testfixtures"}, } diff --git a/pkg/internal/sqlsmith/bulkio.go b/pkg/internal/sqlsmith/bulkio.go index 51940e6ab59e..4cbba8344478 100644 --- a/pkg/internal/sqlsmith/bulkio.go +++ b/pkg/internal/sqlsmith/bulkio.go @@ -106,11 +106,16 @@ func makeBackup(s *Smither) (tree.Statement, bool) { s.bulkBackups[name] = targets s.lock.Unlock() + coinD := tree.DBoolFalse + if s.coin() { + coinD = tree.DBoolTrue + } + return &tree.Backup{ Targets: &targets, To: tree.StringOrPlaceholderOptList{tree.NewStrVal(name)}, AsOf: makeAsOf(s), - Options: tree.BackupOptions{CaptureRevisionHistory: s.coin()}, + Options: tree.BackupOptions{CaptureRevisionHistory: coinD}, }, true } diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 32e460ebccc0..efd7382f03b5 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -417,15 +417,16 @@ func EvalAddSSTable( } else if !ok { break } - for _, rkv := range rangeIter.RangeKeys() { - if err = readWriter.PutRawMVCCRangeKey(rkv.RangeKey, rkv.Value); err != nil { + rangeKeys := rangeIter.RangeKeys() + for _, v := range rangeKeys.Versions { + if err = readWriter.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), v.Value); err != nil { return result.Result{}, err } if sstToReqTS.IsSet() { readWriter.LogLogicalOp(storage.MVCCDeleteRangeOpType, storage.MVCCLogicalOpDetails{ - Key: rkv.RangeKey.StartKey, - EndKey: rkv.RangeKey.EndKey, - Timestamp: rkv.RangeKey.Timestamp, + Key: rangeKeys.Bounds.Key, + EndKey: rangeKeys.Bounds.EndKey, + Timestamp: v.Timestamp, }) } } @@ -517,26 +518,28 @@ func assertSSTContents(sst []byte, sstTimestamp hlc.Timestamp, stats *enginepb.M break } - for _, rkv := range iter.RangeKeys() { - if err := rkv.RangeKey.Validate(); err != nil { + rangeKeys := iter.RangeKeys() + for _, v := range rangeKeys.Versions { + rangeKey := rangeKeys.AsRangeKey(v) + if err := rangeKey.Validate(); err != nil { return errors.NewAssertionErrorWithWrappedErrf(err, "SST contains invalid range key") } - if sstTimestamp.IsSet() && rkv.RangeKey.Timestamp != sstTimestamp { + if sstTimestamp.IsSet() && v.Timestamp != sstTimestamp { return errors.AssertionFailedf( "SST has unexpected timestamp %s (expected %s) for range key %s", - rkv.RangeKey.Timestamp, sstTimestamp, rkv.RangeKey) + v.Timestamp, sstTimestamp, rangeKeys.Bounds) } - value, err := storage.DecodeMVCCValue(rkv.Value) + value, err := storage.DecodeMVCCValue(v.Value) if err != nil { return errors.NewAssertionErrorWithWrappedErrf(err, - "SST contains invalid range key value for range key %s", rkv.RangeKey) + "SST contains invalid range key value for range key %s", rangeKey) } if !value.IsTombstone() { - return errors.AssertionFailedf("SST contains non-tombstone range key %s", rkv.RangeKey) + return errors.AssertionFailedf("SST contains non-tombstone range key %s", rangeKey) } if value.MVCCValueHeader != (enginepb.MVCCValueHeader{}) { return errors.AssertionFailedf("SST contains non-empty MVCC value header for range key %s", - rkv.RangeKey) + rangeKey) } } } diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 05d9189b947b..fa6797d40cbf 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -186,7 +186,7 @@ func TestEvalAddSSTable(t *testing.T) { toReqTS: 1, sst: kvs{pointKV("a", 1, "a1"), rangeKV("c", "d", 2, "")}, expectErr: []string{ - `unexpected timestamp 0.000000002,0 (expected 0.000000001,0) for range key {c-d}/0.000000002,0`, + `unexpected timestamp 0.000000002,0 (expected 0.000000001,0) for range key {c-d}`, `key has suffix "\x00\x00\x00\x00\x00\x00\x00\x02\t", expected "\x00\x00\x00\x00\x00\x00\x00\x01\t"`, }, }, diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 9ecea57c93af..0cc476408354 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -218,9 +218,9 @@ func computeStatsDelta( if ok, err := iter.Valid(); err != nil { return err } else if ok && iter.RangeBounds().Key.Compare(bound) < 0 { - for i, rkv := range iter.RangeKeys() { - keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) - valBytes := int64(len(rkv.Value)) + for i, v := range iter.RangeKeys().Versions { + keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(v.Timestamp)) + valBytes := int64(len(v.Value)) if i == 0 { delta.RangeKeyCount-- keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(bound)) @@ -229,7 +229,7 @@ func computeStatsDelta( delta.RangeValCount-- delta.RangeValBytes -= valBytes delta.GCBytesAge -= (keyBytes + valBytes) * - (delta.LastUpdateNanos/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + (delta.LastUpdateNanos/1e9 - v.Timestamp.WallTime/1e9) } } return nil diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go index 4e454a1001ea..bae99f27b225 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range_test.go @@ -306,8 +306,8 @@ func checkPredicateDeleteRange(t *testing.T, engine storage.Reader, rKeyInfo sto // PredicateDeleteRange should not have written any delete tombstones; // therefore, any range key tombstones in the span should have been // written before the request was issued. - for _, rKey := range iter.RangeKeys() { - require.Equal(t, true, rKey.RangeKey.Timestamp.Less(rKeyInfo.Timestamp)) + for _, v := range iter.RangeKeys().Versions { + require.True(t, v.Timestamp.Less(rKeyInfo.Timestamp)) } continue } @@ -337,13 +337,14 @@ func checkDeleteRangeTombstone( break } require.True(t, ok) - for _, rkv := range iter.RangeKeys() { - if rkv.RangeKey.Timestamp.Equal(rangeKey.Timestamp) { + rangeKeys := iter.RangeKeys() + for _, v := range rangeKeys.Versions { + if v.Timestamp.Equal(rangeKey.Timestamp) { if len(seen.RangeKey.StartKey) == 0 { - seen = rkv.Clone() + seen = rangeKeys.AsRangeKeyValue(v).Clone() } else { - seen.RangeKey.EndKey = rkv.RangeKey.EndKey.Clone() - require.Equal(t, seen.Value, rkv.Value) + seen.RangeKey.EndKey = rangeKeys.Bounds.EndKey.Clone() + require.Equal(t, seen.Value, v.Value) } break } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 61e816d2fc30..9ecd47faa890 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -1315,9 +1315,9 @@ func computeSplitRangeKeyStatsDelta( // contribution of the range key fragmentation. The naïve calculation would be // rhs.EncodedSize() - (keyLen(rhs.EndKey) - keyLen(lhs.EndKey)) // which simplifies to 2 * keyLen(rhs.StartKey) + tsLen(rhs.Timestamp). - for i, rkv := range iter.RangeKeys() { - keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) - valBytes := int64(len(rkv.Value)) + for i, v := range iter.RangeKeys().Versions { + keyBytes := int64(storage.EncodedMVCCTimestampSuffixLength(v.Timestamp)) + valBytes := int64(len(v.Value)) if i == 0 { delta.RangeKeyCount++ keyBytes += 2 * int64(storage.EncodedMVCCKeyPrefixLength(splitKey)) @@ -1325,7 +1325,7 @@ func computeSplitRangeKeyStatsDelta( delta.RangeKeyBytes += keyBytes delta.RangeValCount++ delta.RangeValBytes += valBytes - delta.GCBytesAge += (keyBytes + valBytes) * (nowNanos/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + delta.GCBytesAge += (keyBytes + valBytes) * (nowNanos/1e9 - v.Timestamp.WallTime/1e9) } return delta, nil diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go index f5bf350d07fe..213f799553c8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go @@ -93,12 +93,8 @@ func refreshRange( key := iter.UnsafeKey().Clone() if _, hasRange := iter.HasPointAndRange(); hasRange { - rangeKVs := iter.RangeKeys() - if len(rangeKVs) == 0 { // defensive - return errors.Errorf("expected range key at %s not found", key) - } return roachpb.NewRefreshFailedError(roachpb.RefreshFailedError_REASON_COMMITTED_VALUE, - key.Key, rangeKVs[0].RangeKey.Timestamp) + key.Key, iter.RangeKeys().Versions[0].Timestamp) } if !key.IsValue() { diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 21bbff07c058..71d9ab635d17 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -855,7 +855,8 @@ func processReplicatedRangeTombstones( if !ok { break } - rangeKeys := iter.RangeKeys() + // TODO(erikgrinaker): Rewrite to use MVCCRangeKeyStack. + rangeKeys := iter.RangeKeys().AsRangeKeyValues() if idx := sort.Search(len(rangeKeys), func(i int) bool { return rangeKeys[i].RangeKey.Timestamp.LessEq(gcThreshold) diff --git a/pkg/kv/kvserver/gc/gc_iterator.go b/pkg/kv/kvserver/gc/gc_iterator.go index c657033dcfed..33c2bb5ec768 100644 --- a/pkg/kv/kvserver/gc/gc_iterator.go +++ b/pkg/kv/kvserver/gc/gc_iterator.go @@ -11,7 +11,6 @@ package gc import ( - "sort" "strings" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -193,15 +192,13 @@ func (it *gcIterator) currentRangeTS() hlc.Timestamp { if rangeTombstoneStartKey.Equal(it.cachedRangeTombstoneKey) { return it.cachedRangeTombstoneTS } + it.cachedRangeTombstoneKey = append(it.cachedRangeTombstoneKey[:0], rangeTombstoneStartKey...) - it.cachedRangeTombstoneTS = hlc.Timestamp{} - rangeKeys := it.it.RangeKeys() - if idx := sort.Search(len(rangeKeys), func(i int) bool { - return rangeKeys[i].RangeKey.Timestamp.LessEq(it.threshold) - }); idx < len(rangeKeys) { - it.cachedRangeTombstoneTS = rangeKeys[idx].RangeKey.Timestamp + if v, ok := it.it.RangeKeys().FirstBelow(it.threshold); ok { + it.cachedRangeTombstoneTS = v.Timestamp + } else { + it.cachedRangeTombstoneTS = hlc.Timestamp{} } - it.cachedRangeTombstoneKey = append(it.cachedRangeTombstoneKey[:0], rangeTombstoneStartKey...) return it.cachedRangeTombstoneTS } diff --git a/pkg/kv/kvserver/gc/gc_random_test.go b/pkg/kv/kvserver/gc/gc_random_test.go index 1e6b73216fce..d6300731e8c9 100644 --- a/pkg/kv/kvserver/gc/gc_random_test.go +++ b/pkg/kv/kvserver/gc/gc_random_test.go @@ -466,12 +466,12 @@ func getExpectationsGenerator( // so we will add them to history of current key for analysis. // Bare range tombstones are ignored. if r { - for _, r := range it.RangeKeys() { + for _, r := range it.RangeKeys().AsRangeKeys() { history = append(history, historyItem{ MVCCKeyValue: storage.MVCCKeyValue{ Key: storage.MVCCKey{ - Key: r.RangeKey.StartKey, - Timestamp: r.RangeKey.Timestamp, + Key: r.StartKey, + Timestamp: r.Timestamp, }, Value: nil, }, @@ -603,7 +603,7 @@ func getKeyHistory(t *testing.T, r storage.Reader, key roachpb.Key) string { break } if r && len(result) == 0 { - for _, rk := range it.RangeKeys() { + for _, rk := range it.RangeKeys().AsRangeKeyValues() { result = append(result, fmt.Sprintf("R:%s", rk.RangeKey.String())) } } @@ -622,13 +622,8 @@ func rangeFragmentsFromIt(t *testing.T, it storage.MVCCIterator) [][]storage.MVC if !ok { break } - _, r := it.HasPointAndRange() - if r { - fragments := make([]storage.MVCCRangeKeyValue, len(it.RangeKeys())) - for i, r := range it.RangeKeys() { - fragments[i] = r.Clone() - } - result = append(result, fragments) + if _, hasRange := it.HasPointAndRange(); hasRange { + result = append(result, it.RangeKeys().Clone().AsRangeKeyValues()) } it.NextKey() } diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 29b5145b0def..0cb598f2f0f9 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -1084,20 +1084,20 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) [] _, r := rangeIt.HasPointAndRange() if r { span := rangeIt.RangeBounds() - newKeys := rangeIt.RangeKeys() + newKeys := rangeIt.RangeKeys().AsRangeKeys() if lastEnd.Equal(span.Key) { // Try merging keys by timestamp. var newPartial []storage.MVCCRangeKey i, j := 0, 0 for i < len(newKeys) && j < len(partialRangeKeys) { - switch newKeys[i].RangeKey.Timestamp.Compare(partialRangeKeys[j].Timestamp) { + switch newKeys[i].Timestamp.Compare(partialRangeKeys[j].Timestamp) { case 1: - newPartial = append(newPartial, newKeys[i].RangeKey.Clone()) + newPartial = append(newPartial, newKeys[i].Clone()) i++ case 0: newPartial = append(newPartial, storage.MVCCRangeKey{ StartKey: partialRangeKeys[j].StartKey, - EndKey: newKeys[i].RangeKey.EndKey.Clone(), + EndKey: newKeys[i].EndKey.Clone(), Timestamp: partialRangeKeys[j].Timestamp, }) i++ @@ -1108,7 +1108,7 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) [] } } for ; i < len(newKeys); i++ { - newPartial = append(newPartial, newKeys[i].RangeKey.Clone()) + newPartial = append(newPartial, newKeys[i].Clone()) } for ; j < len(partialRangeKeys); j++ { newPartial = append(newPartial, partialRangeKeys[j].Clone()) @@ -1118,7 +1118,7 @@ func engineData(t *testing.T, r storage.Reader, desc roachpb.RangeDescriptor) [] result = append(result, makeRangeCells(partialRangeKeys)...) partialRangeKeys = make([]storage.MVCCRangeKey, len(newKeys)) for i, rk := range newKeys { - partialRangeKeys[i] = rk.RangeKey.Clone() + partialRangeKeys[i] = rk.Clone() } } lastEnd = span.EndKey.Clone() diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index f64258f941ab..40db0f58796a 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -154,15 +154,15 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err rangeKeysStart = append(rangeKeysStart[:0], rangeBounds.Key...) // Emit events for these MVCC range tombstones, in chronological order. - rangeKeys := i.RangeKeys() - for i := len(rangeKeys) - 1; i >= 0; i-- { + versions := i.RangeKeys().Versions + for i := len(versions) - 1; i >= 0; i-- { var span roachpb.Span a, span.Key = a.Copy(rangeBounds.Key, 0) a, span.EndKey = a.Copy(rangeBounds.EndKey, 0) err := outputFn(&roachpb.RangeFeedEvent{ DeleteRange: &roachpb.RangeFeedDeleteRange{ Span: span, - Timestamp: rangeKeys[i].RangeKey.Timestamp, + Timestamp: versions[i].Timestamp, }, }) if err != nil { @@ -271,8 +271,7 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err // to rangeKeysStart above, because NextIgnoringTime() could reveal // additional MVCC range tombstones below StartTime that cover this // point. We need to find a more performant way to handle this. - if !hasRange || !storage.HasRangeKeyBetween( - i.RangeKeys(), reorderBuf[l].Val.Value.Timestamp, ts) { + if !hasRange || !i.RangeKeys().HasBetween(ts, reorderBuf[l].Val.Value.Timestamp) { // TODO(sumeer): find out if it is deliberate that we are not populating // PrevValue.Timestamp. reorderBuf[l].Val.PrevValue.RawBytes = val diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 6cb45e20a0ef..263ef4fcfb23 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -204,9 +204,9 @@ func (s *testIterator) RangeBounds() roachpb.Span { return roachpb.Span{} } -// RangeTombstones implements SimpleMVCCIterator. -func (s *testIterator) RangeKeys() []storage.MVCCRangeKeyValue { - return []storage.MVCCRangeKeyValue{} +// RangeKeys implements SimpleMVCCIterator. +func (s *testIterator) RangeKeys() storage.MVCCRangeKeyStack { + return storage.MVCCRangeKeyStack{} } func TestInitResolvedTSScan(t *testing.T) { diff --git a/pkg/kv/kvserver/rditer/replica_data_iter.go b/pkg/kv/kvserver/rditer/replica_data_iter.go index 078835b8c467..9435fedf55c2 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter.go @@ -341,7 +341,7 @@ func (ri *ReplicaMVCCDataIterator) UnsafeValue() []byte { // RangeKeys exposes RangeKeys from underlying iterator. See // storage.SimpleMVCCIterator for details. -func (ri *ReplicaMVCCDataIterator) RangeKeys() []storage.MVCCRangeKeyValue { +func (ri *ReplicaMVCCDataIterator) RangeKeys() storage.MVCCRangeKeyStack { return ri.it.RangeKeys() } diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index ffe1bb4ae70d..cc73567cf99e 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -176,19 +176,20 @@ func verifyRDReplicatedOnlyMVCCIter( } } if r { - rks := iter.RangeKeys() - if !rks[0].RangeKey.StartKey.Equal(rangeStart) { + rangeKeys := iter.RangeKeys().Clone() + if !rangeKeys.Bounds.Key.Equal(rangeStart) { + rangeStart = rangeKeys.Bounds.Key.Clone() if !reverse { - for _, rk := range rks { - actualRanges = append(actualRanges, rk.RangeKey.Clone()) + for _, v := range rangeKeys.Versions { + actualRanges = append(actualRanges, rangeKeys.AsRangeKey(v)) } } else { - for i := len(rks) - 1; i >= 0; i-- { - actualRanges = append([]storage.MVCCRangeKey{rks[i].RangeKey.Clone()}, + for i := rangeKeys.Len() - 1; i >= 0; i-- { + actualRanges = append([]storage.MVCCRangeKey{ + rangeKeys.AsRangeKey(rangeKeys.Versions[i])}, actualRanges...) } } - rangeStart = rks[0].RangeKey.StartKey.Clone() } } next() diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index f07d5c5c98d9..44c1c91a6ae7 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -186,7 +186,7 @@ func (i *MVCCIterator) RangeBounds() roachpb.Span { } // RangeKeys implements SimpleMVCCIterator. -func (i *MVCCIterator) RangeKeys() []storage.MVCCRangeKeyValue { +func (i *MVCCIterator) RangeKeys() storage.MVCCRangeKeyStack { return i.i.RangeKeys() } diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 884111c790e7..c30910f6b1f0 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -1996,16 +1996,18 @@ func planProjectionOperators( } leftExpr, rightExpr := t.TypedLeft(), t.TypedRight() if t.Operator.Symbol == treebin.Concat { - // Concat requires special handling. + // Concat requires special handling since it has special rules when + // one of the arguments is an array or a string. We don't have + // native vectorized support for arrays yet, so we don't have to do + // anything extra for them, but we do need to handle the string + // case. leftType, rightType := leftExpr.ResolvedType(), rightExpr.ResolvedType() - if leftType.Family() != rightType.Family() { - // When we have two different types, we perform the STRING - // concatenation. + if t.Op.ReturnType == types.String && leftType.Family() != rightType.Family() { + // This is a special case of the STRING concatenation - we have + // to plan a cast of the non-string type to a STRING. if leftType.Family() == types.StringFamily { - // Need to cast the right expr to the STRING. rightExpr = tree.NewTypedCastExpr(rightExpr, types.String) } else if rightType.Family() == types.StringFamily { - // Need to cast the left expr to the STRING. leftExpr = tree.NewTypedCastExpr(leftExpr, types.String) } else { // This is unexpected. diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index a2add4538cf8..8acf25f4962b 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -389,7 +389,6 @@ func (p *planner) maybeLogStatementInternal( } if telemetryMetrics.maybeUpdateLastEmittedTime(telemetryMetrics.timeNow(), requiredTimeElapsed) { skippedQueries := telemetryMetrics.resetSkippedQueryCount() - databaseName := p.CurrentDatabase() sampledQuery := eventpb.SampledQuery{ CommonSQLExecDetails: execDetails, SkippedQueries: skippedQueries, @@ -397,7 +396,7 @@ func (p *planner) maybeLogStatementInternal( Distribution: p.curPlan.instrumentation.distribution.String(), PlanGist: p.curPlan.instrumentation.planGist.String(), SessionID: p.extendedEvalCtx.SessionID.String(), - Database: databaseName, + Database: p.CurrentDatabase(), StatementID: p.stmt.QueryID.String(), TransactionID: p.txn.ID().String(), StatementFingerprintID: uint64(stmtFingerprintID), @@ -410,10 +409,6 @@ func (p *planner) maybeLogStatementInternal( RowsRead: queryStats.rowsRead, RowsWritten: queryStats.rowsWritten, } - db, _ := p.Descriptors().GetImmutableDatabaseByName(ctx, p.txn, databaseName, tree.DatabaseLookupFlags{Required: true}) - if db != nil { - sampledQuery.DatabaseID = uint32(db.GetID()) - } p.logOperationalEventsOnlyExternally(ctx, eventLogEntry{event: &sampledQuery}) } else { telemetryMetrics.incSkippedQueryCount() diff --git a/pkg/sql/logictest/testdata/logic_test/array b/pkg/sql/logictest/testdata/logic_test/array index 440c78317b78..ba8fe7b9103f 100644 --- a/pkg/sql/logictest/testdata/logic_test/array +++ b/pkg/sql/logictest/testdata/logic_test/array @@ -2263,3 +2263,11 @@ SELECT indkey FROM pg_index WHERE indrelid = (SELECT oid FROM pg_class WHERE rel statement ok RESET distsql_workmem + +# Regression test for incorrect planning in the vectorized engine for +# concatenating a string with a string array (#85295). +query T +SELECT ('foo'::STRING || col::STRING[])::STRING[] FROM (VALUES (ARRAY['bar':::STRING]), (ARRAY['baz':::STRING])) AS t (col); +---- +{foo,bar} +{foo,baz} diff --git a/pkg/sql/logictest/testdata/logic_test/srfs b/pkg/sql/logictest/testdata/logic_test/srfs index 988b2b360a7d..de022092f0a9 100644 --- a/pkg/sql/logictest/testdata/logic_test/srfs +++ b/pkg/sql/logictest/testdata/logic_test/srfs @@ -1248,3 +1248,25 @@ SELECT * FROM unnest(NULL, NULL) query error pq: column reference "unnest" is ambiguous SELECT unnest FROM unnest(array[1,2], array[3,4,5]) + +# Regression test for #58438 - handle the case when unnest outputs a tuple with +# labels. The unnest should not panic. +statement ok +CREATE TABLE t58438(a INT, b INT); + +statement ok +INSERT INTO t58438 VALUES (1, 2), (3, 4), (5, 6); + +query T rowsort +SELECT unnest(ARRAY[t58438.*]) FROM t58438; +---- +(1,2) +(3,4) +(5,6) + +query II rowsort +SELECT (x).* FROM (SELECT unnest(ARRAY[t58438.*]) FROM t58438) v(x); +---- +1 2 +3 4 +5 6 diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index 94217afeb5ba..b96b17439034 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -15,7 +15,7 @@ CREATE TABLE t_implicit_type(a INT PRIMARY KEY, b STRING); statement error pq: return type mismatch in function declared to return int\nDETAIL: Actual return type is record CREATE FUNCTION f() RETURNS INT IMMUTABLE LANGUAGE SQL AS $$ SELECT a, b from t_implicit_type $$ -statement error pq: unimplemented: functions do not currently support \* expressions\nHINT: You have attempted to use a feature that is not yet implemented\.\nSee: https://go\.crdb\.dev/issue-v/10028/v22\.2 +statement error pq: unimplemented: functions do not currently support \* expressions.* CREATE FUNCTION f() RETURNS t_implicit_type IMMUTABLE LANGUAGE SQL AS $$ SELECT * from t_implicit_type $$ # statement ok diff --git a/pkg/sql/opt/bench/bench_test.go b/pkg/sql/opt/bench/bench_test.go index f773b08f2207..c8d2b93e9e04 100644 --- a/pkg/sql/opt/bench/bench_test.go +++ b/pkg/sql/opt/bench/bench_test.go @@ -326,7 +326,7 @@ func init() { // on what each phase includes. func BenchmarkPhases(b *testing.B) { for _, query := range queriesToTest(b) { - h := newHarness(b, query) + h := newHarness(b, query, schemas) b.Run(query.name, func(b *testing.B) { b.Run("Simple", func(b *testing.B) { for _, phase := range SimplePhases { @@ -365,7 +365,7 @@ type harness struct { optimizer xform.Optimizer } -func newHarness(tb testing.TB, query benchQuery) *harness { +func newHarness(tb testing.TB, query benchQuery, schemas []string) *harness { h := &harness{ ctx: context.Background(), semaCtx: tree.MakeSemaContext(), @@ -653,7 +653,7 @@ func queriesToTest(b *testing.B) []benchQuery { func BenchmarkChain(b *testing.B) { for i := 1; i < 20; i++ { q := makeChain(i) - h := newHarness(b, q) + h := newHarness(b, q, schemas) b.Run(q.name, func(b *testing.B) { for i := 0; i < b.N; i++ { h.runSimple(b, q, Explore) @@ -697,3 +697,232 @@ func BenchmarkEndToEnd(b *testing.B) { }) } } + +var slowSchemas = []string{ + ` + CREATE TABLE table64793_1 ( + col1_0 CHAR NOT NULL, col1_1 BOOL NOT NULL, col1_2 REGPROC NOT NULL, + col1_3 REGPROCEDURE NOT NULL, col1_4 TIMETZ NOT NULL, col1_5 FLOAT8 NULL, + col1_6 INT2 NOT NULL, col1_7 BOOL, col1_8 BOX2D NOT NULL, + col1_9 REGNAMESPACE NOT NULL, + PRIMARY KEY ( + col1_8 DESC, col1_9 DESC, col1_4 DESC, col1_1, col1_2 ASC, col1_3 DESC, + col1_0 DESC, col1_6 + ), + col1_10 INT2 NOT NULL AS (col1_6 + 22798:::INT8) VIRTUAL, + FAMILY (col1_4), FAMILY (col1_0, col1_5), FAMILY (col1_1), + FAMILY (col1_8, col1_3, col1_9, col1_7), FAMILY (col1_2), FAMILY (col1_6)) +`, + ` + CREATE TYPE greeting64793 AS ENUM ('hello', 'howdy', 'hi', 'good day', 'morning'); +`, + ` + CREATE TABLE seed64793 ( + _int2 INT2, + _int4 INT4, + _int8 INT8, + _float4 FLOAT4, + _float8 FLOAT8, + _date DATE, + _timestamp TIMESTAMP, + _timestamptz TIMESTAMPTZ, + _interval INTERVAL, + _bool BOOL, + _decimal DECIMAL, + _string STRING, + _bytes BYTES, + _uuid UUID, + _inet INET, + _jsonb JSONB, + _enum greeting64793 + ); +`, + ` + CREATE INDEX on seed64793 (_int8, _float8, _date); +`, + ` + CREATE INVERTED INDEX on seed64793 (_jsonb); +`, + ` + CREATE TABLE table64793_2 ( + col1_0 "char" NOT NULL, col1_1 OID NOT NULL, col1_2 BIT(38) NOT NULL, + col1_3 BIT(18) NOT NULL, col1_4 BYTES NOT NULL, col1_5 INT8 NOT NULL, + col1_6 INTERVAL NOT NULL, col1_7 BIT(33) NOT NULL, col1_8 INTERVAL NULL, + col1_9 GEOMETRY NOT NULL, col1_10 BOOL NOT NULL, col1_11 INT2, + PRIMARY KEY ( + col1_4 ASC, col1_7 DESC, col1_1 ASC, col1_2 ASC, col1_10 ASC, col1_5, + col1_0 ASC, col1_3, col1_6 + ), + UNIQUE ( + col1_8 DESC, col1_11, col1_3 DESC, col1_7, col1_6 DESC, col1_4 ASC, + col1_1 DESC + ) + ); +`, + ` + CREATE TABLE table64793_3 ( + col2_0 NAME NOT NULL, col2_1 TIMETZ NOT NULL, + PRIMARY KEY (col2_0 ASC, col2_1), + col2_2 STRING NOT NULL AS (lower(col2_0)) VIRTUAL, + UNIQUE (col2_0 DESC, col2_2 DESC, col2_1) + WHERE (table64793_3.col2_2 > e'\U00002603':::STRING) + OR (table64793_3.col2_0 != '"':::STRING), + UNIQUE (col2_1 ASC, col2_2, col2_0), + UNIQUE (col2_0 DESC,col2_1, col2_2), + INDEX (col2_1 DESC), + UNIQUE (col2_2 DESC, col2_0 ASC) + WHERE table64793_3.col2_2 = '"':::STRING + ); +`, + ` + CREATE TABLE table64793_4 ( + col2_0 NAME NOT NULL, col2_1 TIMETZ NOT NULL, col3_2 REGPROC NOT NULL, + col3_3 "char", col3_4 BOX2D, col3_5 INT8 NULL, col3_6 TIMESTAMP NOT NULL, + col3_7 FLOAT8, col3_8 INT4 NULL, col3_9 INET NULL, col3_10 UUID NOT NULL, + col3_11 UUID NULL, col3_12 INT2 NOT NULL, col3_13 BIT(34), + col3_14 REGPROCEDURE NULL, col3_15 FLOAT8 NULL, + PRIMARY KEY ( + col2_0 ASC, col2_1, col3_11 DESC, col3_13, col3_6, col3_3 DESC, + col3_15 ASC, col3_2 ASC, col3_4 ASC, col3_9 DESC, col3_12 ASC, + col3_8 ASC, col3_5, col3_14 ASC + ), + UNIQUE (col3_2, col3_8 ASC) + WHERE ((((table64793_4.col3_5 < 0:::INT8) + AND (table64793_4.col3_3 != '':::STRING)) + AND (table64793_4.col2_1 < '00:00:00+15:59:00':::TIMETZ)) + AND (table64793_4.col3_12 > 0:::INT8)) + AND (table64793_4.col3_15 <= 1.7976931348623157e+308:::FLOAT8), + UNIQUE (col3_10 DESC, col3_3 ASC, col2_1 DESC, col3_9 ASC) + ); +`, +} + +var slowQueries = [...]benchQuery{ + // 1. The first long-running query taken from #64793. + // 2. The most recent long-running query from #64793 (as of July 2022). + { + name: "slow-query-1", + query: ` + WITH with_186941 (col_1103773, col_1103774) AS ( + SELECT + * + FROM + ( + VALUES + ('clvl', 3 :: INT2), + ( + 'n', + ( + SELECT + tab_455284.col1_6 AS col_1103772 + FROM + table64793_1@[0] AS tab_455284 + ORDER BY + tab_455284.col1_2 DESC, + tab_455284.col1_1 DESC + LIMIT + 1 ::: INT8 + ) + ), + (NULL, 6736 ::: INT8) + ) AS tab_455285 (col_1103773, col_1103774) + ), + with_186942 (col_1103775) AS ( + SELECT + * + FROM + ( + VALUES + ('yk'), + (NULL) + ) AS tab_455286 (col_1103775) + ) + SELECT + 0 ::: OID AS col_1103776, + ( + (-32244820164.24410487)::: DECIMAL :: DECIMAL + tab_455291.col1_10 :: INT8 + ):: DECIMAL AS col_1103777, + tab_455287._bool AS col_1103778 + FROM + with_186942 AS cte_ref_54113, + seed64793@[0] AS tab_455287 + JOIN seed64793 AS tab_455288 + JOIN seed64793 AS tab_455289 ON (tab_455288._int8) = (tab_455289._int8) + AND (tab_455288._date) = (tab_455289._date) + AND (tab_455288._float8) = (tab_455289._float8) + JOIN table64793_1@[0] AS tab_455290 + JOIN table64793_1@primary AS tab_455291 + JOIN table64793_1@[0] AS tab_455295 + JOIN seed64793 AS tab_455296 + JOIN seed64793 AS tab_455297 ON (tab_455296._int8) = (tab_455297._int8) + AND (tab_455296._date) = (tab_455297._date) ON (tab_455295.col1_5) = (tab_455297._float8) + AND (tab_455295.col1_5) = (tab_455296._float8) + AND (tab_455295.col1_5) = (tab_455297._float8) + AND (tab_455295.col1_5) = (tab_455297._float8) ON (tab_455291.col1_2) = (tab_455295.tableoid) + AND (tab_455291.col1_7) = (tab_455295.col1_1) ON (tab_455290.col1_2) = (tab_455291.col1_9) + AND (tab_455290.col1_7) = (tab_455291.col1_7) ON (tab_455289._float8) = (tab_455296._float8) ON (tab_455287._float4) = (tab_455290.col1_5) + AND (tab_455287.tableoid) = (tab_455295.col1_9) + AND (tab_455287._bool) = (tab_455295.col1_7); + `, + args: []interface{}{}, + }, + { + name: "slow-query-2", + query: ` + WITH with_121707 (col_692430) AS ( + SELECT + * + FROM + ( + VALUES + ( + (-0.19099748134613037)::: FLOAT8 + ), + (0.9743397235870361 ::: FLOAT8), + ( + (-1.6944892406463623)::: FLOAT8 + ) + ) AS tab_297691 (col_692430) + ) + SELECT + '-35 years -11 mons -571 days -08:18:57.001029' ::: INTERVAL AS col_692441 + FROM + table64793_2@table64793_2_col1_8_col1_11_col1_3_col1_7_col1_6_col1_4_col1_1_key AS tab_297692 + JOIN table64793_3@table64793_3_col2_0_col2_1_col2_2_key AS tab_297693 + JOIN table64793_2@[0] AS tab_297694 + JOIN seed64793@seed64793__int8__float8__date_idx AS tab_297695 + RIGHT JOIN table64793_3@[0] AS tab_297696 + JOIN table64793_4@table64793_4_col3_10_col3_3_col2_1_col3_9_key AS tab_297697 ON (tab_297696.col2_0) = (tab_297697.col3_3) CROSS + JOIN table64793_4@[0] AS tab_297698 + JOIN table64793_3 AS tab_297699 ON (tab_297698.col2_0) = (tab_297699.col2_0) ON TRUE + JOIN table64793_4@[0] AS tab_297700 ON (tab_297697.col3_12) = (tab_297700.col3_8) ON (tab_297694.tableoid) = (tab_297695.tableoid) + AND (tab_297694.col1_5) = (tab_297698.col3_8) + AND (tab_297694.tableoid) = (tab_297698.col3_2) + AND (tab_297694.col1_5) = (tab_297697.col3_12) ON (tab_297693.col2_2) = (tab_297700.col3_3) + AND (tab_297693.col2_1) = (tab_297698.col2_1) + AND (tab_297693.tableoid) = (tab_297699.tableoid) + AND (tab_297693.col2_1) = (tab_297697.col2_1) + AND (tab_297693.tableoid) = (tab_297694.col1_1) + AND (tab_297693.col2_2) = (tab_297695._string) + AND (tab_297693.col2_2) = (tab_297696.col2_0) + AND (tab_297693.col2_2) = (tab_297698.col3_3) ON (tab_297692.col1_11) = (tab_297694.col1_11) + ORDER BY + tab_297695._enum DESC + LIMIT + 57 ::: INT8; + `, + args: []interface{}{}, + }, +} + +func BenchmarkSlowQueries(b *testing.B) { + for _, query := range slowQueries { + h := newHarness(b, query, slowSchemas) + h.evalCtx.SessionData().ReorderJoinsLimit = 8 + b.Run(query.name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + h.runSimple(b, query, Explore) + } + }) + } +} diff --git a/pkg/sql/opt/exec/explain/plan_gist_factory.go b/pkg/sql/opt/exec/explain/plan_gist_factory.go index 307c642be0ec..19807090fcf0 100644 --- a/pkg/sql/opt/exec/explain/plan_gist_factory.go +++ b/pkg/sql/opt/exec/explain/plan_gist_factory.go @@ -415,8 +415,10 @@ func (f *PlanGistFactory) decodeScanParams() exec.ScanParams { if l > 0 { idxConstraint = new(constraint.Constraint) idxConstraint.Spans.Alloc(l) - var sp constraint.Span - idxConstraint.Spans.Append(&sp) + for i := 0; i < l; i++ { + var sp constraint.Span + idxConstraint.Spans.Append(&sp) + } } var invertedConstraint inverted.Spans diff --git a/pkg/sql/opt/exec/explain/testdata/gists b/pkg/sql/opt/exec/explain/testdata/gists index 253ceccd2d9f..582973507c30 100644 --- a/pkg/sql/opt/exec/explain/testdata/gists +++ b/pkg/sql/opt/exec/explain/testdata/gists @@ -1119,3 +1119,17 @@ explain(shape): • explain explain(gist): • explain + +# Regression test for #85324 +explain-plan-gist +AgHyAQIA//8HHgAAByoFKiHyAQAA +---- +• update +│ table: ? +│ set +│ +└── • render + │ + └── • scan + table: ?@? + spans: 15 spans diff --git a/pkg/sql/opt/optbuilder/testdata/srfs b/pkg/sql/opt/optbuilder/testdata/srfs index 88fb3b3be849..612356e2614f 100644 --- a/pkg/sql/opt/optbuilder/testdata/srfs +++ b/pkg/sql/opt/optbuilder/testdata/srfs @@ -965,3 +965,38 @@ with &1 (data) │ └── filters (true) └── aggregations └── count-rows [as=count_rows:8] + +# Regression test for #58438 - handle the case when unnest outputs a tuple with +# labels. The unnest should not panic. +exec-ddl +CREATE TABLE t58438(a INT, b INT); +---- + +build +SELECT unnest(ARRAY[t58438.*]) FROM t58438; +---- +project + ├── columns: unnest:6 + └── project-set + ├── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 tableoid:5 unnest:6 + ├── scan t58438 + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 tableoid:5 + └── zip + └── unnest(ARRAY[((a:1, b:2) AS a, b)]) + +build +SELECT (x).* FROM (SELECT unnest(ARRAY[t58438.*]) FROM t58438) v(x); +---- +project + ├── columns: a:7 b:8 + ├── project + │ ├── columns: unnest:6 + │ └── project-set + │ ├── columns: t58438.a:1 t58438.b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 tableoid:5 unnest:6 + │ ├── scan t58438 + │ │ └── columns: t58438.a:1 t58438.b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 tableoid:5 + │ └── zip + │ └── unnest(ARRAY[((t58438.a:1, t58438.b:2) AS a, b)]) + └── projections + ├── (unnest:6).a [as=a:7] + └── (unnest:6).b [as=b:8] diff --git a/pkg/sql/opt/optbuilder/util.go b/pkg/sql/opt/optbuilder/util.go index ddcc1bdf29ce..f6e31fbaaee3 100644 --- a/pkg/sql/opt/optbuilder/util.go +++ b/pkg/sql/opt/optbuilder/util.go @@ -244,18 +244,29 @@ func (b *Builder) projectColumn(dst *scopeColumn, src *scopeColumn) { dst.id = src.id } -// shouldCreateDefaultColumn decides if we need to create a default -// column and default label for a function expression. -// Returns true if the function's return type is not an empty tuple and -// doesn't declare any tuple labels. +// shouldCreateDefaultColumn decides if we need to create a default column and +// default label for a function expression. Returns true if the function's +// return type is not an empty tuple and doesn't declare any tuple labels. func (b *Builder) shouldCreateDefaultColumn(texpr tree.TypedExpr) bool { if texpr.ResolvedType() == types.EmptyTuple { // This is only to support crdb_internal.unary_table(). return false } - // We need to create a default column with a default name when - // the function return type doesn't declare any return labels. + if funcExpr, ok := texpr.(*tree.FuncExpr); ok { + if funcExpr.Func.FunctionReference.(*tree.FunctionDefinition).Name == "unnest" { + // Special case for unnest functions: we should create a default column in + // the case when there is one input argument, since this implies there + // will be one output column. This is necessary because the type of the + // single column output by unnest in this case may be a tuple with labels, + // which breaks the assumption made below. + return len(funcExpr.Exprs) == 1 + } + } + + // We need to create a default column with a default name when the function + // return type doesn't declare any return labels. This logic assumes that any + // SRF that has a labeled tuple as a return type returns multiple columns. return len(texpr.ResolvedType().TupleLabels()) == 0 } diff --git a/pkg/sql/opt/props/BUILD.bazel b/pkg/sql/opt/props/BUILD.bazel index 30727befc506..ab49da07e567 100644 --- a/pkg/sql/opt/props/BUILD.bazel +++ b/pkg/sql/opt/props/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "cardinality.go", "col_stats_map.go", + "equiv_set.go", "func_dep.go", "histogram.go", "logical.go", @@ -43,6 +44,7 @@ go_test( srcs = [ "cardinality_test.go", "col_stats_map_test.go", + "equiv_set_test.go", "func_dep_rand_test.go", "func_dep_test.go", "histogram_test.go", diff --git a/pkg/sql/opt/props/equiv_set.go b/pkg/sql/opt/props/equiv_set.go new file mode 100644 index 000000000000..2bf4b1b760b4 --- /dev/null +++ b/pkg/sql/opt/props/equiv_set.go @@ -0,0 +1,112 @@ +// Copyright 2022 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 props + +import "github.com/cockroachdb/cockroach/pkg/sql/opt" + +// EquivSet describes a set of equivalence groups of columns. It can answer +// queries about which columns are equivalent to one another. Equivalence groups +// are always non-empty and disjoint. +// +// TODO(drewk): incorporate EquivSets into FuncDepSets. +type EquivSet struct { + buf [equalityBufferSize]opt.ColSet + groups []opt.ColSet +} + +const equalityBufferSize = 1 + +// NewEquivSet returns a new equality set with a starting capacity of one +// equivalence group. This optimizes for the common case when only one +// equivalence group is stored. +func NewEquivSet() EquivSet { + set := EquivSet{} + set.groups = set.buf[:0] + return set +} + +// Reset prepares the EquivSet for reuse, maintaining references to any +// allocated slice memory. +func (eq *EquivSet) Reset() { + for i := range eq.groups { + // Release any references to the large portion of ColSets. + eq.groups[i] = opt.ColSet{} + } + eq.groups = eq.groups[:0] +} + +// Add adds the given equivalent columns to the EquivSet. If possible, the +// columns are added to an existing group. Otherwise, a new one is created. +func (eq *EquivSet) Add(equivCols opt.ColSet) { + // Attempt to add the equivalence to an existing group. + for i := range eq.groups { + if eq.groups[i].Intersects(equivCols) { + if equivCols.SubsetOf(eq.groups[i]) { + // No-op + return + } + eq.groups[i].UnionWith(equivCols) + eq.tryMergeGroups(i) + return + } + } + // Make a new equivalence group. + eq.groups = append(eq.groups, equivCols.Copy()) +} + +// AddFromFDs adds all equivalence relations from the given FuncDepSet to the +// EquivSet. +func (eq *EquivSet) AddFromFDs(fdset *FuncDepSet) { + for i := range fdset.deps { + fd := &fdset.deps[i] + if fd.equiv { + eq.Add(fd.from.Union(fd.to)) + } + } +} + +// AreColsEquiv indicates whether the given columns are equivalent. +func (eq *EquivSet) AreColsEquiv(left, right opt.ColumnID) bool { + for i := range eq.groups { + if eq.groups[i].Contains(left) { + return eq.groups[i].Contains(right) + } + if eq.groups[i].Contains(right) { + return eq.groups[i].Contains(left) + } + } + return false +} + +// tryMergeGroups attempts to merge the equality group at the given index with +// any of the *following* groups. If a group can be merged, it is removed after +// its columns are added to the given group. +func (eq *EquivSet) tryMergeGroups(idx int) { + for i := idx + 1; i < len(eq.groups); i++ { + if eq.groups[idx].Intersects(eq.groups[i]) { + eq.groups[idx].UnionWith(eq.groups[i]) + eq.groups[i] = eq.groups[len(eq.groups)-1] + eq.groups[len(eq.groups)-1] = opt.ColSet{} + eq.groups = eq.groups[:len(eq.groups)-1] + } + } +} + +func (eq *EquivSet) String() string { + ret := "[" + for i := range eq.groups { + if i > 0 { + ret += ", " + } + ret += eq.groups[i].String() + } + return ret + "]" +} diff --git a/pkg/sql/opt/props/equiv_set_test.go b/pkg/sql/opt/props/equiv_set_test.go new file mode 100644 index 000000000000..a48cef9d70e7 --- /dev/null +++ b/pkg/sql/opt/props/equiv_set_test.go @@ -0,0 +1,120 @@ +// Copyright 2022 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 props + +import ( + "fmt" + "math/rand" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/opt" + "github.com/stretchr/testify/require" +) + +func TestEquivSet_Rand(t *testing.T) { + const maxCol = 512 + const maxCols = 256 + const numIterations = 8 + const chanceUseExisting = 0.5 + + // Testing oracle + testOracle := make(map[opt.ColumnID]opt.ColSet) + addToOracle := func(left, right opt.ColumnID) { + leftSet, rightSet := testOracle[left], testOracle[right] + if leftSet.Contains(right) || rightSet.Contains(left) { + return + } + leftSet.Add(right) + rightSet.Add(left) + leftSet.UnionWith(rightSet) + rightSet.UnionWith(leftSet) + closure := func(left, right opt.ColSet) { + for leftCol, leftOk := left.Next(0); leftOk; leftCol, leftOk = left.Next(leftCol + 1) { + addTo := testOracle[leftCol] + addTo.UnionWith(right) + testOracle[leftCol] = addTo + } + } + closure(leftSet, rightSet) + closure(rightSet, leftSet) + testOracle[left] = leftSet + testOracle[right] = rightSet + } + areColsEquivOracle := func(left, right opt.ColumnID) bool { + return testOracle[left].Contains(right) + } + + // Utility funcs + getNthCol := func(set opt.ColSet, n int) (col opt.ColumnID) { + // Assume n <= set.Len() + var i int + var ok bool + for col, ok = set.Next(0); i < n && ok; col, ok = set.Next(col + 1) { + i++ + } + return col + } + getRandValueForCol := func(colsUsed opt.ColSet, useExisting bool) opt.ColumnID { + if useExisting { + return getNthCol(colsUsed, rand.Intn(colsUsed.Len())) + } + return opt.ColumnID(rand.Intn(maxCol) + 1) + } + makeError := func(expected bool) string { + if expected { + return "expected columns to be equivalent, but weren't" + } + return "expected columns not to be equivalent, but were" + } + + equivSet := NewEquivSet() + for numCols := 2; numCols <= maxCols; numCols = numCols << 1 { + for i := 0; i < numIterations; i++ { + var colsUsed opt.ColSet + var fds FuncDepSet + t.Run(fmt.Sprintf("cols=%d/iteration=%d", numCols, i), func(t *testing.T) { + for c := 0; c < numCols; c++ { + useExistingLeft := rand.Float64() < chanceUseExisting + useExistingRight := rand.Float64() < chanceUseExisting + if colsUsed.Len() <= 1 { + useExistingLeft, useExistingRight = false, false + } + leftCol := getRandValueForCol(colsUsed, useExistingLeft) + rightCol := getRandValueForCol(colsUsed, useExistingRight) + for leftCol == rightCol { + rightCol = getRandValueForCol(colsUsed, useExistingRight) + } + addToOracle(leftCol, rightCol) + equivSet.Add(opt.MakeColSet(leftCol, rightCol)) + fds.AddEquivalency(leftCol, rightCol) + colsUsed.Add(leftCol) + colsUsed.Add(rightCol) + } + var fromFDEquivSet EquivSet + fromFDEquivSet.AddFromFDs(&fds) + for leftCol, leftOk := colsUsed.Next(0); leftOk; leftCol, leftOk = colsUsed.Next(leftCol + 1) { + for rightCol, rightOk := colsUsed.Next(0); rightOk; rightCol, rightOk = colsUsed.Next(rightCol + 1) { + if leftCol == rightCol { + continue + } + expected := areColsEquivOracle(leftCol, rightCol) + actual := equivSet.AreColsEquiv(leftCol, rightCol) + require.Equal(t, expected, actual, makeError(expected)) + actualFromFDs := fromFDEquivSet.AreColsEquiv(leftCol, rightCol) + require.Equal(t, expected, actualFromFDs, makeError(expected)) + } + } + }) + testOracle = make(map[opt.ColumnID]opt.ColSet) + equivSet.Reset() + } + } +} diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index 04cfa53f01be..b20c29924c73 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -772,7 +772,7 @@ func (c *coster) computeSelectCost(sel *memo.SelectExpr, required *physical.Requ inputRowCount = math.Min(inputRowCount, required.LimitHint/selectivity) } - filterSetup, filterPerRow := c.computeFiltersCost(sel.Filters, util.FastIntMap{}) + filterSetup, filterPerRow := c.computeFiltersCost(sel.Filters, util.FastIntSet{}) cost := memo.Cost(inputRowCount) * filterPerRow cost += filterSetup return cost @@ -827,25 +827,17 @@ func (c *coster) computeHashJoinCost(join memo.RelExpr) memo.Cost { // pressure and the possibility of spilling to disk. cost += c.rowBufferCost(rightRowCount) - // Compute filter cost. Fetch the equality columns so they can be - // ignored later. + // Compute filter cost. Fetch the indices of the filters that will be used in + // the join, since they will not add to the cost and should be skipped. on := join.Child(2).(*memo.FiltersExpr) - leftEq, rightEq, _ := memo.ExtractJoinEqualityColumns( - join.Child(0).(memo.RelExpr).Relational().OutputCols, - join.Child(1).(memo.RelExpr).Relational().OutputCols, - *on, - ) - // Generate a quick way to lookup if two columns are join equality - // columns. We add in both directions because we don't know which way - // the equality filters will be defined. - eqMap := util.FastIntMap{} - for i := range leftEq { - left := int(leftEq[i]) - right := int(rightEq[i]) - eqMap.Set(left, right) - eqMap.Set(right, left) - } - filterSetup, filterPerRow := c.computeFiltersCost(*on, eqMap) + leftCols := join.Child(0).(memo.RelExpr).Relational().OutputCols + rightCols := join.Child(1).(memo.RelExpr).Relational().OutputCols + var filtersToSkip util.FastIntSet + _, _, toSkip := memo.ExtractJoinEqualityColumns(leftCols, rightCols, *on) + for _, idx := range toSkip { + filtersToSkip.Add(idx) + } + filterSetup, filterPerRow := c.computeFiltersCost(*on, filtersToSkip) cost += filterSetup // Add the CPU cost of emitting the rows. @@ -882,7 +874,7 @@ func (c *coster) computeMergeJoinCost(join *memo.MergeJoinExpr) memo.Cost { // smaller right side is preferred to the symmetric join. cost := memo.Cost(0.9*leftRowCount+1.1*rightRowCount) * cpuCostFactor - filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntMap{}) + filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntSet{}) cost += filterSetup // Add the CPU cost of emitting the rows. @@ -996,7 +988,7 @@ func (c *coster) computeIndexLookupJoinCost( perLookupCost += lookupExprCost(join) cost := memo.Cost(lookupCount) * perLookupCost - filterSetup, filterPerRow := c.computeFiltersCost(on, util.FastIntMap{}) + filterSetup, filterPerRow := c.computeFiltersCost(on, util.FastIntSet{}) cost += filterSetup // Each lookup might retrieve many rows; add the IO cost of retrieving the @@ -1075,7 +1067,7 @@ func (c *coster) computeInvertedJoinCost( perLookupCost *= 5 cost := memo.Cost(lookupCount) * perLookupCost - filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntMap{}) + filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntSet{}) cost += filterSetup // Each lookup might retrieve many rows; add the IO cost of retrieving the @@ -1108,28 +1100,21 @@ func (c *coster) computeExprCost(expr opt.Expr) memo.Cost { // computeFiltersCost returns the setup and per-row cost of executing // a filter. Callers of this function should add setupCost and multiply // perRowCost by the number of rows expected to be filtered. +// +// filtersToSkip identifies the indices of filters that should be skipped, +// because they do not add to the cost. This can happen when a condition still +// exists in the filters even though it is handled by the join. func (c *coster) computeFiltersCost( - filters memo.FiltersExpr, eqMap util.FastIntMap, + filters memo.FiltersExpr, filtersToSkip util.FastIntSet, ) (setupCost, perRowCost memo.Cost) { // Add a base perRowCost so that callers do not need to have their own // base per-row cost. perRowCost += cpuCostFactor for i := range filters { - f := &filters[i] - if f.Condition.Op() == opt.EqOp { - eq := f.Condition.(*memo.EqExpr) - leftVar, lOk := eq.Left.(*memo.VariableExpr) - rightVar, rOk := eq.Right.(*memo.VariableExpr) - if lOk && rOk { - val, ok := eqMap.Get(int(leftVar.Col)) - if ok && val == int(rightVar.Col) { - // Equality filters on some joins are still in - // filters, while others have already removed - // them. They do not cost anything. - continue - } - } + if filtersToSkip.Contains(i) { + continue } + f := &filters[i] perRowCost += c.computeExprCost(f.Condition) // Add a constant "setup" cost per ON condition to account for the fact that // the rowsProcessed estimate alone cannot effectively discriminate between @@ -1156,7 +1141,7 @@ func (c *coster) computeZigzagJoinCost(join *memo.ZigzagJoinExpr) memo.Cost { scanCost := c.rowScanCost(join, join.LeftTable, join.LeftIndex, leftCols, join.Relational().Stats) scanCost += c.rowScanCost(join, join.RightTable, join.RightIndex, rightCols, join.Relational().Stats) - filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntMap{}) + filterSetup, filterPerRow := c.computeFiltersCost(join.On, util.FastIntSet{}) // Double the cost of emitting rows as well as the cost of seeking rows, // given two indexes will be accessed. diff --git a/pkg/sql/opt/xform/join_funcs.go b/pkg/sql/opt/xform/join_funcs.go index 5f8bc938439d..012bf688c63e 100644 --- a/pkg/sql/opt/xform/join_funcs.go +++ b/pkg/sql/opt/xform/join_funcs.go @@ -93,10 +93,14 @@ func (c *CustomFuncs) GenerateMergeJoins( return } - var colToEq util.FastIntMap - for i := range leftEq { - colToEq.Set(int(leftEq[i]), i) - colToEq.Set(int(rightEq[i]), i) + getEqCols := func(col opt.ColumnID) (left, right opt.ColumnID) { + // Assume that col is in either leftEq or rightEq. + for eqIdx := 0; eqIdx < len(leftEq); eqIdx++ { + if leftEq[eqIdx] == col || rightEq[eqIdx] == col { + return leftEq[eqIdx], rightEq[eqIdx] + } + } + panic(errors.AssertionFailedf("failed to find eqIdx for merge join")) } var remainingFilters memo.FiltersExpr @@ -115,8 +119,7 @@ func (c *CustomFuncs) GenerateMergeJoins( merge.RightOrdering.Columns = make([]props.OrderingColumnChoice, 0, n) addCol := func(col opt.ColumnID, descending bool) { - eqIdx, _ := colToEq.Get(int(col)) - l, r := leftEq[eqIdx], rightEq[eqIdx] + l, r := getEqCols(col) merge.LeftEq = append(merge.LeftEq, opt.MakeOrderingColumn(l, descending)) merge.RightEq = append(merge.RightEq, opt.MakeOrderingColumn(r, descending)) merge.LeftOrdering.AppendCol(l, descending) diff --git a/pkg/sql/opt/xform/join_order_builder.go b/pkg/sql/opt/xform/join_order_builder.go index d0bea26af806..9be542c616e8 100644 --- a/pkg/sql/opt/xform/join_order_builder.go +++ b/pkg/sql/opt/xform/join_order_builder.go @@ -311,6 +311,10 @@ type JoinOrderBuilder struct { // once does not exceed the session limit. joinCount int + // equivs is an EquivSet used to keep track of equivalence relations when + // assembling filters. + equivs props.EquivSet + onReorderFunc OnReorderFunc onAddJoinFunc OnAddJoinFunc @@ -328,6 +332,7 @@ func (jb *JoinOrderBuilder) Init(f *norm.Factory, evalCtx *eval.Context) { plans: make(map[vertexSet]memo.RelExpr), onReorderFunc: jb.onReorderFunc, onAddJoinFunc: jb.onAddJoinFunc, + equivs: props.NewEquivSet(), } } @@ -501,9 +506,9 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) { return } - var fds props.FuncDepSet - fds.AddEquivFrom(&jb.plans[s1].Relational().FuncDeps) - fds.AddEquivFrom(&jb.plans[s2].Relational().FuncDeps) + jb.equivs.Reset() + jb.equivs.AddFromFDs(&jb.plans[s1].Relational().FuncDeps) + jb.equivs.AddFromFDs(&jb.plans[s2].Relational().FuncDeps) // Gather all inner edges that connect the left and right relation sets. var innerJoinFilters memo.FiltersExpr @@ -515,7 +520,7 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) { // Ensure that this edge forms a valid connection between the two sets. See // the checkNonInnerJoin and checkInnerJoin comments for more information. if e.checkInnerJoin(s1, s2) { - if areFiltersRedundant(&fds, e.filters) { + if areFiltersRedundant(&jb.equivs, e.filters) { // Avoid adding redundant filters. continue } @@ -524,7 +529,9 @@ func (jb *JoinOrderBuilder) addJoins(s1, s2 vertexSet) { // s2, any other edges that apply will also be part of that original join. joinIsRedundant = e.joinIsRedundant(s1, s2) } - getEquivFDs(&fds, e.filters) + for j := range e.filters { + jb.equivs.AddFromFDs(&e.filters[j].ScalarProps().FuncDeps) + } innerJoinFilters = append(innerJoinFilters, e.filters...) addInnerJoin = true } @@ -722,7 +729,7 @@ func (jb *JoinOrderBuilder) addJoin( // areFiltersRedundant returns true if the given FiltersExpr contains a single // equality filter that is already represented by the given FuncDepSet. -func areFiltersRedundant(fds *props.FuncDepSet, filters memo.FiltersExpr) bool { +func areFiltersRedundant(equivs *props.EquivSet, filters memo.FiltersExpr) bool { if len(filters) != 1 { return false } @@ -735,15 +742,7 @@ func areFiltersRedundant(fds *props.FuncDepSet, filters memo.FiltersExpr) bool { if !ok1 || !ok2 { return false } - return fds.AreColsEquiv(var1.Col, var2.Col) -} - -// getEquivFDs adds all equivalencies from the given filters to the given -// FuncDepSet. -func getEquivFDs(fds *props.FuncDepSet, filters memo.FiltersExpr) { - for i := range filters { - fds.AddEquivFrom(&filters[i].ScalarProps().FuncDeps) - } + return equivs.AreColsEquiv(var1.Col, var2.Col) } // addToGroup adds a join of the given type and with the given inputs to the diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f822ad983ac2..2867c500cae8 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -2972,11 +2972,23 @@ backup_options: } | REVISION_HISTORY { - $$.val = &tree.BackupOptions{CaptureRevisionHistory: true} + $$.val = &tree.BackupOptions{CaptureRevisionHistory: tree.MakeDBool(true)} + } +| REVISION_HISTORY '=' a_expr + { + $$.val = &tree.BackupOptions{CaptureRevisionHistory: $3.expr()} } | DETACHED { - $$.val = &tree.BackupOptions{Detached: true} + $$.val = &tree.BackupOptions{Detached: tree.MakeDBool(true)} + } +| DETACHED '=' TRUE + { + $$.val = &tree.BackupOptions{Detached: tree.MakeDBool(true)} + } +| DETACHED '=' FALSE + { + $$.val = &tree.BackupOptions{Detached: tree.MakeDBool(false)} } | KMS '=' string_or_placeholder_opt_list { diff --git a/pkg/sql/parser/testdata/backup_restore b/pkg/sql/parser/testdata/backup_restore index fb6fd33b9a20..a9409e1a45ef 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -315,36 +315,36 @@ BACKUP DATABASE _ TO ($1, $2) INCREMENTAL FROM 'baz' -- identifiers removed parse BACKUP foo TO 'bar' WITH ENCRYPTION_PASSPHRASE = 'secret', revision_history ---- -BACKUP TABLE foo TO 'bar' WITH revision_history, encryption_passphrase = '*****' -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH revision_history, encryption_passphrase = '*****' -- fully parenthesized -BACKUP TABLE foo TO '_' WITH revision_history, encryption_passphrase = '*****' -- literals removed -BACKUP TABLE _ TO 'bar' WITH revision_history, encryption_passphrase = '*****' -- identifiers removed -BACKUP TABLE foo TO 'bar' WITH revision_history, encryption_passphrase = 'secret' -- passwords exposed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, encryption_passphrase = '*****' -- normalized! +BACKUP TABLE (foo) TO ('bar') WITH revision_history = (true), encryption_passphrase = '*****' -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = _, encryption_passphrase = '*****' -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = true, encryption_passphrase = '*****' -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, encryption_passphrase = 'secret' -- passwords exposed parse BACKUP foo TO 'bar' WITH KMS = ('foo', 'bar'), revision_history ---- -BACKUP TABLE foo TO 'bar' WITH revision_history, kms = ('foo', 'bar') -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH revision_history, kms = (('foo'), ('bar')) -- fully parenthesized -BACKUP TABLE foo TO '_' WITH revision_history, kms = ('_', '_') -- literals removed -BACKUP TABLE _ TO 'bar' WITH revision_history, kms = ('foo', 'bar') -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, kms = ('foo', 'bar') -- normalized! +BACKUP TABLE (foo) TO ('bar') WITH revision_history = (true), kms = (('foo'), ('bar')) -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = _, kms = ('_', '_') -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = true, kms = ('foo', 'bar') -- identifiers removed parse BACKUP foo TO 'bar' WITH OPTIONS (detached, ENCRYPTION_PASSPHRASE = 'secret', revision_history) ---- -BACKUP TABLE foo TO 'bar' WITH revision_history, encryption_passphrase = '*****', detached -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH revision_history, encryption_passphrase = '*****', detached -- fully parenthesized -BACKUP TABLE foo TO '_' WITH revision_history, encryption_passphrase = '*****', detached -- literals removed -BACKUP TABLE _ TO 'bar' WITH revision_history, encryption_passphrase = '*****', detached -- identifiers removed -BACKUP TABLE foo TO 'bar' WITH revision_history, encryption_passphrase = 'secret', detached -- passwords exposed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, encryption_passphrase = '*****', detached -- normalized! +BACKUP TABLE (foo) TO ('bar') WITH revision_history = (true), encryption_passphrase = '*****', detached -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = _, encryption_passphrase = '*****', detached -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = true, encryption_passphrase = '*****', detached -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, encryption_passphrase = 'secret', detached -- passwords exposed parse BACKUP foo TO 'bar' WITH OPTIONS (detached, KMS = ('foo', 'bar'), revision_history) ---- -BACKUP TABLE foo TO 'bar' WITH revision_history, detached, kms = ('foo', 'bar') -- normalized! -BACKUP TABLE (foo) TO ('bar') WITH revision_history, detached, kms = (('foo'), ('bar')) -- fully parenthesized -BACKUP TABLE foo TO '_' WITH revision_history, detached, kms = ('_', '_') -- literals removed -BACKUP TABLE _ TO 'bar' WITH revision_history, detached, kms = ('foo', 'bar') -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, detached, kms = ('foo', 'bar') -- normalized! +BACKUP TABLE (foo) TO ('bar') WITH revision_history = (true), detached, kms = (('foo'), ('bar')) -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = _, detached, kms = ('_', '_') -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = true, detached, kms = ('foo', 'bar') -- identifiers removed parse BACKUP TENANT 36 TO 'bar' @@ -740,10 +740,18 @@ RESTORE TENANT 123 FROM REPLICATION STREAM FROM $1 -- identifiers removed parse BACKUP TABLE foo TO 'bar' WITH revision_history, detached ---- -BACKUP TABLE foo TO 'bar' WITH revision_history, detached -BACKUP TABLE (foo) TO ('bar') WITH revision_history, detached -- fully parenthesized -BACKUP TABLE foo TO '_' WITH revision_history, detached -- literals removed -BACKUP TABLE _ TO 'bar' WITH revision_history, detached -- identifiers removed +BACKUP TABLE foo TO 'bar' WITH revision_history = true, detached -- normalized! +BACKUP TABLE (foo) TO ('bar') WITH revision_history = (true), detached -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = _, detached -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = true, detached -- identifiers removed + +parse +BACKUP TABLE foo TO 'bar' WITH revision_history = $1, detached +---- +BACKUP TABLE foo TO 'bar' WITH revision_history = $1, detached +BACKUP TABLE (foo) TO ('bar') WITH revision_history = ($1), detached -- fully parenthesized +BACKUP TABLE foo TO '_' WITH revision_history = $1, detached -- literals removed +BACKUP TABLE _ TO 'bar' WITH revision_history = $1, detached -- identifiers removed parse RESTORE TABLE foo FROM 'bar' WITH skip_missing_foreign_keys, skip_missing_sequences, detached @@ -765,18 +773,43 @@ HINT: try \h BACKUP error BACKUP foo TO 'bar' WITH revision_history, revision_history ---- -at or near "revision_history": syntax error: revision_history option specified multiple times +at or near "EOF": syntax error: revision_history option specified multiple times DETAIL: source SQL: BACKUP foo TO 'bar' WITH revision_history, revision_history - ^ + ^ error BACKUP foo TO 'bar' WITH detached, revision_history, detached ---- -at or near "detached": syntax error: detached option specified multiple times +at or near "EOF": syntax error: detached option specified multiple times DETAIL: source SQL: BACKUP foo TO 'bar' WITH detached, revision_history, detached - ^ + ^ + +error +BACKUP foo TO 'bar' WITH revision_history=false, revision_history, detached +---- +at or near ",": syntax error: revision_history option specified multiple times +DETAIL: source SQL: +BACKUP foo TO 'bar' WITH revision_history=false, revision_history, detached + ^ + +error +BACKUP foo TO 'bar' WITH detached=true, revision_history, detached=true +---- +at or near "true": syntax error: detached option specified multiple times +DETAIL: source SQL: +BACKUP foo TO 'bar' WITH detached=true, revision_history, detached=true + ^ + +error +BACKUP foo TO 'bar' WITH detached=$1, revision_history +---- +at or near "1": syntax error +DETAIL: source SQL: +BACKUP foo TO 'bar' WITH detached=$1, revision_history + ^ +HINT: try \h BACKUP error RESTORE foo FROM 'bar' WITH key1, key2 = 'value' diff --git a/pkg/sql/parser/testdata/create_schedule b/pkg/sql/parser/testdata/create_schedule index bbf152f24848..5c712a9c24b9 100644 --- a/pkg/sql/parser/testdata/create_schedule +++ b/pkg/sql/parser/testdata/create_schedule @@ -41,23 +41,23 @@ CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' RECURRING '@daily' FULL BACK parse CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' ---- -CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' -CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('bar') WITH revision_history RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized -CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '_' WITH revision_history RECURRING '_' FULL BACKUP '_' -- literals removed -CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- normalized! +CREATE SCHEDULE FOR BACKUP TABLE (foo), (bar), (buz) INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP TABLE foo, bar, buz INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' -- literals removed +CREATE SCHEDULE FOR BACKUP TABLE _, _, _ INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' -- identifiers removed parse CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' ---- -CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' -CREATE SCHEDULE FOR BACKUP INTO ('bar') WITH revision_history RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS foo = ('bar') -- fully parenthesized -CREATE SCHEDULE FOR BACKUP INTO '_' WITH revision_history RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS foo = '_' -- literals removed -CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'bar' -- identifiers removed +CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS foo = 'bar' -- normalized! +CREATE SCHEDULE FOR BACKUP INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS foo = ('bar') -- fully parenthesized +CREATE SCHEDULE FOR BACKUP INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS foo = '_' -- literals removed +CREATE SCHEDULE FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'bar' -- identifiers removed parse CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' ---- -CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' -CREATE SCHEDULE IF NOT EXISTS ('baz') FOR BACKUP INTO ('bar') WITH revision_history RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS first_run = ('now') -- fully parenthesized -CREATE SCHEDULE IF NOT EXISTS '_' FOR BACKUP INTO '_' WITH revision_history RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS first_run = '_' -- literals removed -CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'now' -- identifiers removed +CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS first_run = 'now' -- normalized! +CREATE SCHEDULE IF NOT EXISTS ('baz') FOR BACKUP INTO ('bar') WITH revision_history = (true) RECURRING ('@daily') FULL BACKUP ('@weekly') WITH SCHEDULE OPTIONS first_run = ('now') -- fully parenthesized +CREATE SCHEDULE IF NOT EXISTS '_' FOR BACKUP INTO '_' WITH revision_history = _ RECURRING '_' FULL BACKUP '_' WITH SCHEDULE OPTIONS first_run = '_' -- literals removed +CREATE SCHEDULE IF NOT EXISTS 'baz' FOR BACKUP INTO 'bar' WITH revision_history = true RECURRING '@daily' FULL BACKUP '@weekly' WITH SCHEDULE OPTIONS _ = 'now' -- identifiers removed diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index 24fb361598e6..510f8f283bba 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -88,6 +88,7 @@ type PlanHookState interface { DistSQLPlanner() *DistSQLPlanner LeaseMgr() *lease.Manager TypeAsString(ctx context.Context, e tree.Expr, op string) (func() (string, error), error) + TypeAsBool(ctx context.Context, e tree.Expr, op string) (func() (bool, error), error) TypeAsStringArray(ctx context.Context, e tree.Exprs, op string) (func() ([]string, error), error) TypeAsStringOpts( ctx context.Context, opts tree.KVOptions, optsValidate map[string]KVStringOptValidate, diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index d3e43b465309..c8569a921d03 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -653,6 +653,46 @@ func (p *planner) makeStringEvalFn(typedE tree.TypedExpr) func() (bool, string, } } +// TypeAsBool enforces (not hints) that the given expression typechecks as a +// string and returns a function that can be called to get the string value +// during (planNode).Start. +func (p *planner) TypeAsBool( + ctx context.Context, e tree.Expr, op string, +) (func() (bool, error), error) { + typedE, err := tree.TypeCheckAndRequire(ctx, e, &p.semaCtx, types.Bool, op) + if err != nil { + return nil, err + } + evalFn := p.makeBoolEvalFn(typedE) + return func() (bool, error) { + isNull, b, err := evalFn() + if err != nil { + return false, err + } + if isNull { + return false, errors.Errorf("expected string, got NULL") + } + return b, nil + }, nil +} + +func (p *planner) makeBoolEvalFn(typedE tree.TypedExpr) func() (bool, bool, error) { + return func() (bool, bool, error) { + d, err := eval.Expr(p.EvalContext(), typedE) + if err != nil { + return false, false, err + } + if d == tree.DNull { + return true, false, nil + } + b, ok := d.(*tree.DBool) + if !ok { + return false, false, errors.Errorf("failed to cast %T to bool", d) + } + return false, bool(*b), nil + } +} + // KVStringOptValidate indicates the requested validation of a TypeAsStringOpts // option. type KVStringOptValidate string diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index 3f58c5e52822..6a68a755148a 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -40,9 +40,9 @@ const ( // BackupOptions describes options for the BACKUP execution. type BackupOptions struct { - CaptureRevisionHistory bool + CaptureRevisionHistory Expr EncryptionPassphrase Expr - Detached bool + Detached *DBool EncryptionKMSURI StringOrPlaceholderOptList IncrementalStorage StringOrPlaceholderOptList } @@ -244,8 +244,9 @@ func (o *BackupOptions) Format(ctx *FmtCtx) { } addSep = true } - if o.CaptureRevisionHistory { - ctx.WriteString("revision_history") + if o.CaptureRevisionHistory != nil { + ctx.WriteString("revision_history = ") + ctx.FormatNode(o.CaptureRevisionHistory) addSep = true } @@ -259,7 +260,7 @@ func (o *BackupOptions) Format(ctx *FmtCtx) { } } - if o.Detached { + if o.Detached == DBoolTrue { maybeAddSep() ctx.WriteString("detached") } @@ -280,8 +281,8 @@ func (o *BackupOptions) Format(ctx *FmtCtx) { // CombineWith merges other backup options into this backup options struct. // An error is returned if the same option merged multiple times. func (o *BackupOptions) CombineWith(other *BackupOptions) error { - if o.CaptureRevisionHistory { - if other.CaptureRevisionHistory { + if o.CaptureRevisionHistory != nil { + if other.CaptureRevisionHistory != nil { return errors.New("revision_history option specified multiple times") } } else { @@ -294,8 +295,8 @@ func (o *BackupOptions) CombineWith(other *BackupOptions) error { return errors.New("encryption_passphrase specified multiple times") } - if o.Detached { - if other.Detached { + if o.Detached != nil { + if other.Detached != nil { return errors.New("detached option specified multiple times") } } else { diff --git a/pkg/sql/sem/tree/testdata/pretty/backup.align-deindent.golden.short b/pkg/sql/sem/tree/testdata/pretty/backup.align-deindent.golden.short index c13f618b381c..692781e68089 100644 --- a/pkg/sql/sem/tree/testdata/pretty/backup.align-deindent.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/backup.align-deindent.golden.short @@ -8,6 +8,6 @@ AS OF SYSTEM TIME '-10s' INCREMENTAL FROM 'gs://acme-co-backup/database-bank-2017-03-27-weekly', 'gs://acme-co-backup/database-bank-2017-03-28-nightly' - WITH revision_history + WITH revision_history = true diff --git a/pkg/sql/sem/tree/testdata/pretty/backup.align-only.golden.short b/pkg/sql/sem/tree/testdata/pretty/backup.align-only.golden.short index c13f618b381c..692781e68089 100644 --- a/pkg/sql/sem/tree/testdata/pretty/backup.align-only.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/backup.align-only.golden.short @@ -8,6 +8,6 @@ AS OF SYSTEM TIME '-10s' INCREMENTAL FROM 'gs://acme-co-backup/database-bank-2017-03-27-weekly', 'gs://acme-co-backup/database-bank-2017-03-28-nightly' - WITH revision_history + WITH revision_history = true diff --git a/pkg/sql/sem/tree/testdata/pretty/backup.ref.golden.short b/pkg/sql/sem/tree/testdata/pretty/backup.ref.golden.short index 7c4ec63d0c4d..e5d65d42f3ec 100644 --- a/pkg/sql/sem/tree/testdata/pretty/backup.ref.golden.short +++ b/pkg/sql/sem/tree/testdata/pretty/backup.ref.golden.short @@ -13,6 +13,6 @@ INCREMENTAL FROM 'gs://acme-co-backup/database-bank-2017-03-27-weekly', 'gs://acme-co-backup/database-bank-2017-03-28-nightly' WITH - revision_history + revision_history = true diff --git a/pkg/sql/sem/tree/testdata/pretty/backup.sql b/pkg/sql/sem/tree/testdata/pretty/backup.sql index ab103ec89142..0176117d0104 100644 --- a/pkg/sql/sem/tree/testdata/pretty/backup.sql +++ b/pkg/sql/sem/tree/testdata/pretty/backup.sql @@ -2,4 +2,4 @@ BACKUP DATABASE bank TO 'gs://acme-co-backup/database-bank-2017-03-29-nightly' AS OF SYSTEM TIME '-10s' INCREMENTAL FROM 'gs://acme-co-backup/database-bank-2017-03-27-weekly', 'gs://acme-co-backup/database-bank-2017-03-28-nightly' -WITH revision_history +WITH revision_history = true diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index ed15c6ad1f96..f1b485ae15f9 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -12,7 +12,6 @@ package sql import ( "context" - gosql "database/sql" "fmt" "math" "regexp" @@ -97,14 +96,10 @@ func TestTelemetryLogging(t *testing.T) { var sessionID string var databaseName string - var dbID uint32 db := sqlutils.MakeSQLRunner(sqlDB) - conn := db.DB.(*gosql.DB) - db.QueryRow(t, `SHOW session_id`).Scan(&sessionID) db.QueryRow(t, `SHOW database`).Scan(&databaseName) - dbID = sqlutils.QueryDatabaseID(t, conn, databaseName) db.Exec(t, `SET application_name = 'telemetry-logging-test'`) db.Exec(t, `SET CLUSTER SETTING sql.telemetry.query_sampling.enabled = true;`) db.Exec(t, "CREATE TABLE t();") @@ -345,9 +340,6 @@ func TestTelemetryLogging(t *testing.T) { if !strings.Contains(e.Message, "\"Database\":\""+databaseName+"\"") { t.Errorf("expected to find Database: %s", databaseName) } - if !strings.Contains(e.Message, "\"DatabaseID\":"+strconv.Itoa(int(dbID))) { - t.Errorf("expected to find DatabaseID: %v", dbID) - } stmtFingerprintID := roachpb.ConstructStatementFingerprintID(tc.queryNoConstants, false, true, databaseName) if !strings.Contains(e.Message, "\"StatementFingerprintID\":"+strconv.FormatUint(uint64(stmtFingerprintID), 10)) { t.Errorf("expected to find StatementFingerprintID: %v", stmtFingerprintID) diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 2e64be956a8e..3d9c5da1cacc 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -168,13 +168,13 @@ type SimpleMVCCIterator interface { // empty span if there are none. The returned keys are only valid until the // next iterator call. RangeBounds() roachpb.Span - // RangeKeys returns all range keys (with different timestamps) at the current - // key position, or an empty list if there are none. When at a point key, it - // will return all range keys overlapping that point key. The keys are only - // valid until the next iterator operation. For details on range keys, see - // comment on SimpleMVCCIterator, or this tech note: + // RangeKeys returns a stack of all range keys (with different timestamps) at + // the current key position. When at a point key, it will return all range + // keys overlapping that point key. The stack is only valid until the next + // iterator operation. For details on range keys, see comment on + // SimpleMVCCIterator, or this tech note: // https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md - RangeKeys() []MVCCRangeKeyValue + RangeKeys() MVCCRangeKeyStack } // IteratorStats is returned from {MVCCIterator,EngineIterator}.Stats. diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index db816062077c..4a4a77d4b424 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -2312,8 +2312,8 @@ func scanRangeKeys(t *testing.T, r Reader) []MVCCRangeKeyValue { if !ok { break } - for _, rangeKey := range iter.RangeKeys() { - rangeKeys = append(rangeKeys, rangeKey.Clone()) + for _, rkv := range iter.RangeKeys().AsRangeKeyValues() { + rangeKeys = append(rangeKeys, rkv.Clone()) } } return rangeKeys @@ -2373,8 +2373,8 @@ func scanIter(t *testing.T, iter SimpleMVCCIterator) []interface{} { hasPoint, hasRange := iter.HasPointAndRange() if hasRange { if bounds := iter.RangeBounds(); !bounds.Key.Equal(prevRangeStart) { - for _, rk := range iter.RangeKeys() { - keys = append(keys, rk.Clone()) + for _, rkv := range iter.RangeKeys().AsRangeKeyValues() { + keys = append(keys, rkv.Clone()) } prevRangeStart = bounds.Key.Clone() } diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 3adfa3458c37..35f4cf723f14 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -934,9 +934,9 @@ func (i *intentInterleavingIter) RangeBounds() roachpb.Span { } // RangeKeys implements SimpleMVCCIterator. -func (i *intentInterleavingIter) RangeKeys() []MVCCRangeKeyValue { +func (i *intentInterleavingIter) RangeKeys() MVCCRangeKeyStack { if _, hasRange := i.HasPointAndRange(); !hasRange { - return []MVCCRangeKeyValue{} + return MVCCRangeKeyStack{} } return i.iter.RangeKeys() } diff --git a/pkg/storage/multi_iterator.go b/pkg/storage/multi_iterator.go index 0f4c2f00561a..1612a8e7008f 100644 --- a/pkg/storage/multi_iterator.go +++ b/pkg/storage/multi_iterator.go @@ -106,7 +106,7 @@ func (f *multiIterator) RangeBounds() roachpb.Span { } // RangeKeys implements SimpleMVCCIterator. -func (f *multiIterator) RangeKeys() []MVCCRangeKeyValue { +func (f *multiIterator) RangeKeys() MVCCRangeKeyStack { panic("not implemented") } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 705ddc85bde9..043211baa74c 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -974,7 +974,7 @@ func mvccGetMetadata( // If we land on a (bare) range key, step to look for a colocated point key. if hasRange && !hasPoint { - rkTimestamp := iter.RangeKeys()[0].RangeKey.Timestamp + rkTimestamp := iter.RangeKeys().Versions[0].Timestamp iter.Next() if ok, err = iter.Valid(); err != nil { @@ -1011,10 +1011,10 @@ func mvccGetMetadata( // metadata), or the point version's timestamp if it was a tombstone. if hasRange { rangeKeys := iter.RangeKeys() - if rkv, ok := FirstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + if v, ok := rangeKeys.FirstAbove(unsafeKey.Timestamp); ok { meta.Deleted = true - meta.Timestamp = rangeKeys[0].RangeKey.Timestamp.ToLegacyTimestamp() - keyLastSeen := rkv.RangeKey.Timestamp + meta.Timestamp = rangeKeys.Versions[0].Timestamp.ToLegacyTimestamp() + keyLastSeen := v.Timestamp if unsafeVal.IsTombstone() { keyLastSeen = unsafeKey.Timestamp } @@ -1665,7 +1665,7 @@ func mvccPutInternal( // We must now be on a point key, but it may be covered by an // existing MVCC range tombstone. If it isn't, account for it. _, hasRange := iter.HasPointAndRange() - if !hasRange || iter.RangeKeys()[0].RangeKey.Timestamp.Less(prevUnsafeKey.Timestamp) { + if !hasRange || iter.RangeKeys().Versions[0].Timestamp.Less(prevUnsafeKey.Timestamp) { prevValRaw := iter.UnsafeValue() prevVal, err := DecodeMVCCValue(prevValRaw) if err != nil { @@ -2568,16 +2568,16 @@ func MVCCPredicateDeleteRange( // TODO (msbutler): cache the range keys while the range bounds remain // constant, since iter.RangeKeys() is expensive. Manual caching may not be necessary if // https://github.com/cockroachdb/cockroach/issues/84379 lands. - rangeKeys := iter.RangeKeys() - if endTime.LessEq(rangeKeys[0].RangeKey.Timestamp) { - return false, false, false, roachpb.NewWriteTooOldError(endTime, - rangeKeys[0].RangeKey.Timestamp.Next(), k.Key.Clone()) + newestRangeKey := iter.RangeKeys().Versions[0].Timestamp + if endTime.LessEq(newestRangeKey) { + return false, false, false, roachpb.NewWriteTooOldError( + endTime, newestRangeKey.Next(), k.Key.Clone()) } if !hasPointKey { // landed on bare range key. return true, false, true, nil } - if k.Timestamp.Less(rangeKeys[0].RangeKey.Timestamp) { + if k.Timestamp.Less(newestRangeKey) { // The latest range tombstone shadows the point key; ok to continue run. return true, false, true, nil } @@ -2872,12 +2872,11 @@ func MVCCDeleteRangeUsingTombstone( if hasRange { // Check if we've encountered a new range key stack. if rangeBounds := iter.RangeBounds(); !rangeBounds.EndKey.Equal(prevRangeEnd) { - newest := iter.RangeKeys()[0].RangeKey + newest := iter.RangeKeys().Versions[0].Timestamp // Check for conflict with newer range key. - if timestamp.LessEq(newest.Timestamp) { - return roachpb.NewWriteTooOldError( - timestamp, newest.Timestamp.Next(), newest.StartKey.Clone()) + if timestamp.LessEq(newest) { + return roachpb.NewWriteTooOldError(timestamp, newest.Next(), rangeBounds.Key.Clone()) } if ms != nil { @@ -2899,7 +2898,7 @@ func MVCCDeleteRangeUsingTombstone( msDelta.RangeKeyBytes += int64(EncodedMVCCTimestampSuffixLength(timestamp)) msDelta.RangeValCount++ msDelta.RangeValBytes += int64(len(valueRaw)) - msDelta.GCBytesAge -= (timestamp.WallTime/1e9 - newest.Timestamp.WallTime/1e9) * + msDelta.GCBytesAge -= (timestamp.WallTime/1e9 - newest.WallTime/1e9) * int64(EncodedMVCCKeyPrefixLength(rangeBounds.Key)+ EncodedMVCCKeyPrefixLength(rangeBounds.EndKey)) } @@ -2944,10 +2943,10 @@ func MVCCDeleteRangeUsingTombstone( if ms != nil { // fragmentRangeKeys adjusts ms to fragment an existing range key stack // at the given split point. - fragmentRangeKeys := func(rangeKeys []MVCCRangeKeyValue, splitKey roachpb.Key) { - for i, rkv := range rangeKeys { - keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) - valBytes := int64(len(rkv.Value)) + fragmentRangeKeys := func(rangeKeys MVCCRangeKeyStack, splitKey roachpb.Key) { + for i, v := range rangeKeys.Versions { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(v.Timestamp)) + valBytes := int64(len(v.Value)) if i == 0 { msDelta.RangeKeyCount++ keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(splitKey)) @@ -2955,37 +2954,29 @@ func MVCCDeleteRangeUsingTombstone( msDelta.RangeKeyBytes += keyBytes msDelta.RangeValCount++ msDelta.RangeValBytes += valBytes - msDelta.GCBytesAge += (keyBytes + valBytes) * (timestamp.WallTime/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + msDelta.GCBytesAge += (keyBytes + valBytes) * (timestamp.WallTime/1e9 - v.Timestamp.WallTime/1e9) } } // maybeMergeRangeKeys adjusts ms to merge two abutting range key stacks if // they have the same timestamps and values. It assumes the lhs end key // equals the rhs start key, and that they are in descending order. - maybeMergeRangeKeys := func(lhs, rhs []MVCCRangeKeyValue) { - if len(lhs) != len(rhs) || len(lhs) == 0 { + maybeMergeRangeKeys := func(lhs, rhs MVCCRangeKeyStack) { + if !lhs.CanMergeRight(rhs) { return } - for i, l := range lhs { - if !l.RangeKey.Timestamp.Equal(rhs[i].RangeKey.Timestamp) { - return - } else if !bytes.Equal(l.Value, rhs[i].Value) { - return - } - } - mergeKey := rhs[0].RangeKey.StartKey - for i, rkv := range lhs { - keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) - valBytes := int64(len(rkv.Value)) + for i, v := range lhs.Versions { + keyBytes := int64(EncodedMVCCTimestampSuffixLength(v.Timestamp)) + valBytes := int64(len(v.Value)) if i == 0 { msDelta.RangeKeyCount-- - keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(mergeKey)) + keyBytes += 2 * int64(EncodedMVCCKeyPrefixLength(rhs.Bounds.Key)) } msDelta.RangeKeyBytes -= keyBytes msDelta.RangeValCount-- msDelta.RangeValBytes -= valBytes msDelta.GCBytesAge -= (keyBytes + valBytes) * - (timestamp.WallTime/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + (timestamp.WallTime/1e9 - v.Timestamp.WallTime/1e9) } } @@ -3005,16 +2996,16 @@ func MVCCDeleteRangeUsingTombstone( case 1: // fragment fragmentRangeKeys(iter.RangeKeys(), startKey) case 0: // merge - lhs := iter.RangeKeys() - for i := range lhs { - lhs[i] = lhs[i].Clone() + lhs := iter.RangeKeys().Clone() + rhs := MVCCRangeKeyStack{ + Bounds: rangeKey.Bounds(), + Versions: MVCCRangeKeyVersions{{Timestamp: rangeKey.Timestamp, Value: valueRaw}}, } - rhs := []MVCCRangeKeyValue{{RangeKey: rangeKey, Value: valueRaw}} iter.SeekGE(MVCCKey{Key: startKey}) if ok, err := iter.Valid(); err != nil { return err } else if ok { - rhs = append(rhs, iter.RangeKeys()...) + rhs.Versions = append(rhs.Versions, iter.RangeKeys().Versions...) } maybeMergeRangeKeys(lhs, rhs) } @@ -3040,16 +3031,16 @@ func MVCCDeleteRangeUsingTombstone( case -1: // fragment fragmentRangeKeys(iter.RangeKeys(), endKey) case 0: // merge - lhs := []MVCCRangeKeyValue{{RangeKey: rangeKey, Value: valueRaw}} - rhs := iter.RangeKeys() - for i := range rhs { - rhs[i] = rhs[i].Clone() + lhs := MVCCRangeKeyStack{ + Bounds: rangeKey.Bounds(), + Versions: MVCCRangeKeyVersions{{Timestamp: rangeKey.Timestamp, Value: valueRaw}}, } + rhs := iter.RangeKeys().Clone() iter.SeekLT(MVCCKey{Key: endKey}) if ok, err := iter.Valid(); err != nil { return err } else if ok { - lhs = append(lhs, iter.RangeKeys()...) + lhs.Versions = append(lhs.Versions, iter.RangeKeys().Versions...) } maybeMergeRangeKeys(lhs, rhs) } @@ -3558,7 +3549,7 @@ type iterForKeyVersions interface { UnsafeKey() MVCCKey UnsafeValue() []byte ValueProto(msg protoutil.Message) error - RangeKeys() []MVCCRangeKeyValue + RangeKeys() MVCCRangeKeyStack } // separatedIntentAndVersionIter is an implementation of iterForKeyVersions @@ -3634,7 +3625,7 @@ func (s *separatedIntentAndVersionIter) HasPointAndRange() (bool, bool) { return hasPoint, hasRange } -func (s *separatedIntentAndVersionIter) RangeKeys() []MVCCRangeKeyValue { +func (s *separatedIntentAndVersionIter) RangeKeys() MVCCRangeKeyStack { return s.mvccIter.RangeKeys() } @@ -4038,7 +4029,7 @@ func mvccResolveWriteIntent( } else if valid { if hasPoint, hasRange := iter.HasPointAndRange(); hasPoint { if unsafeKey := iter.UnsafeKey(); unsafeKey.Key.Equal(oldKey.Key) { - if !hasRange || iter.RangeKeys()[0].RangeKey.Timestamp.Less(unsafeKey.Timestamp) { + if !hasRange || iter.RangeKeys().Versions[0].Timestamp.Less(unsafeKey.Timestamp) { unsafeValRaw := iter.UnsafeValue() prevVal, prevValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) if !prevValOK && err == nil { @@ -4149,8 +4140,8 @@ func mvccResolveWriteIntent( // synthesize a point tombstone at the lowest range tombstone covering it. // This is where the point key ceases to exist, contributing to GCBytesAge. if len(unsafeNextValueRaw) > 0 { - if rk, found := FirstRangeKeyAbove(iter.RangeKeys(), unsafeNextKey.Timestamp); found { - unsafeNextKey.Timestamp = rk.RangeKey.Timestamp + if v, found := iter.RangeKeys().FirstAbove(unsafeNextKey.Timestamp); found { + unsafeNextKey.Timestamp = v.Timestamp unsafeNextValueRaw = []byte{} } } @@ -4608,18 +4599,15 @@ func MVCCGarbageCollect( if ms != nil { // We need to iterate ranges only to compute GCBytesAge if we are updating // stats. + // + // TODO(erikgrinaker): Rewrite to use MVCCRangeKeyStack. if _, hasRange := iter.HasPointAndRange(); hasRange && !lastRangeTombstoneStart.Equal(iter.RangeBounds().Key) { rangeKeys := iter.RangeKeys() - newLen := len(rangeKeys) - if cap(rangeTombstoneTss) < newLen { - rangeTombstoneTss = make([]hlc.Timestamp, newLen) - } else { - rangeTombstoneTss = rangeTombstoneTss[:newLen] - } - for i, rkv := range rangeKeys { - rangeTombstoneTss[i] = rkv.RangeKey.Timestamp + lastRangeTombstoneStart = append(lastRangeTombstoneStart[:0], rangeKeys.Bounds.Key...) + rangeTombstoneTss = rangeTombstoneTss[:0] + for _, v := range rangeKeys.Versions { + rangeTombstoneTss = append(rangeTombstoneTss, v.Timestamp) } - lastRangeTombstoneStart = append(lastRangeTombstoneStart[:0], rangeKeys[0].RangeKey.StartKey...) } else if !hasRange { lastRangeTombstoneStart = lastRangeTombstoneStart[:0] rangeTombstoneTss = rangeTombstoneTss[:0] @@ -4825,8 +4813,9 @@ func MVCCGarbageCollectRangeKeys( break } + // TODO(erikgrinaker): Rewrite to use MVCCRangeKeyStack. bounds := iter.RangeBounds() - unsafeRangeKeys := iter.RangeKeys() + unsafeRangeKeys := iter.RangeKeys().AsRangeKeyValues() // Check if preceding range tombstone is adjacent to GC'd one. If we // started iterating too early, just skip to next key. If boundaries @@ -5158,8 +5147,8 @@ func ComputeStatsForRangeWithVisitors( // reverse chronological order and use this variable to keep track // of the point in time at which the current key begins to age. var accrueGCAgeNanos int64 + var rangeKeys MVCCRangeKeyStack mvccEndKey := MakeMVCCMetadataKey(end) - rangeKeys := []MVCCRangeKeyValue{} for iter.SeekGE(MakeMVCCMetadataKey(start)); ; iter.Next() { if ok, err := iter.Valid(); err != nil { @@ -5173,9 +5162,9 @@ func ComputeStatsForRangeWithVisitors( if hasRange { if rangeStart := iter.RangeBounds().Key; !rangeStart.Equal(prevRangeStart) { prevRangeStart = append(prevRangeStart[:0], rangeStart...) - rangeKeys = iter.RangeKeys() + rangeKeys = iter.RangeKeys().Clone() - for i, rkv := range rangeKeys { + for i, v := range rangeKeys.Versions { // Only the top-most fragment contributes the key and its bounds, but // all versions contribute timestamps and values. // @@ -5183,28 +5172,27 @@ func ComputeStatsForRangeWithVisitors( // though it is actually variable-length, likely for historical // reasons. But for range keys we may as well use the actual // variable-length encoded size. - keyBytes := int64(EncodedMVCCTimestampSuffixLength(rkv.RangeKey.Timestamp)) - valBytes := int64(len(rkv.Value)) + keyBytes := int64(EncodedMVCCTimestampSuffixLength(v.Timestamp)) + valBytes := int64(len(v.Value)) if i == 0 { ms.RangeKeyCount++ - keyBytes += int64(EncodedMVCCKeyPrefixLength(rkv.RangeKey.StartKey) + - EncodedMVCCKeyPrefixLength(rkv.RangeKey.EndKey)) + keyBytes += int64(EncodedMVCCKeyPrefixLength(rangeKeys.Bounds.Key) + + EncodedMVCCKeyPrefixLength(rangeKeys.Bounds.EndKey)) } ms.RangeKeyBytes += keyBytes ms.RangeValCount++ ms.RangeValBytes += valBytes - ms.GCBytesAge += (keyBytes + valBytes) * - (nowNanos/1e9 - rkv.RangeKey.Timestamp.WallTime/1e9) + ms.GCBytesAge += (keyBytes + valBytes) * (nowNanos/1e9 - v.Timestamp.WallTime/1e9) if rangeKeyVisitor != nil { - if err := rangeKeyVisitor(rkv); err != nil { + if err := rangeKeyVisitor(rangeKeys.AsRangeKeyValue(v)); err != nil { return enginepb.MVCCStats{}, err } } } } - } else if len(rangeKeys) > 0 { - rangeKeys = rangeKeys[:0] + } else if !rangeKeys.IsEmpty() { + rangeKeys = MVCCRangeKeyStack{} } if !hasPoint { @@ -5247,8 +5235,8 @@ func ComputeStatsForRangeWithVisitors( // only take them into account for versioned values. var nextRangeTombstone hlc.Timestamp if isValue { - if rkv, ok := FirstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { - nextRangeTombstone = rkv.RangeKey.Timestamp + if v, ok := rangeKeys.FirstAbove(unsafeKey.Timestamp); ok { + nextRangeTombstone = v.Timestamp } } @@ -5435,7 +5423,7 @@ func MVCCExportToSST( var rows RowCounter var curKey roachpb.Key // only used if exportAllRevisions var resumeKey MVCCKey - var rangeKeys []MVCCRangeKeyValue + var rangeKeys MVCCRangeKeyStack var rangeKeysEnd roachpb.Key var rangeKeysSize int64 @@ -5464,7 +5452,7 @@ func MVCCExportToSST( // We could be truncated in the middle of a point key version series, which // would require adding on a \0 byte via Key.Next(), so let's assume that. maxSize := rangeKeysSize - if s := maxSize + int64(len(rangeKeys)*(len(resumeKey)-len(rangeKeysEnd)+1)); s > maxSize { + if s := maxSize + int64(rangeKeys.Len()*(len(resumeKey)-len(rangeKeysEnd)+1)); s > maxSize { maxSize = s } return maxSize @@ -5527,37 +5515,43 @@ func MVCCExportToSST( // comparisons. Pebble should expose an API to cheaply detect range key // changes. if len(rangeKeysEnd) > 0 && bytes.Compare(unsafeKey.Key, rangeKeysEnd) >= 0 { - for _, rkv := range rangeKeys { - mvccValue, ok, err := tryDecodeSimpleMVCCValue(rkv.Value) + for _, v := range rangeKeys.Versions { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(v.Value) if !ok && err == nil { - mvccValue, err = decodeExtendedMVCCValue(rkv.Value) + mvccValue, err = decodeExtendedMVCCValue(v.Value) } if err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, - "decoding mvcc value %s", rkv.Value) + "decoding mvcc value %s", v.Value) } // Export only the inner roachpb.Value, not the MVCCValue header. mvccValue = MVCCValue{Value: mvccValue.Value} - if err := sstWriter.PutMVCCRangeKey(rkv.RangeKey, mvccValue); err != nil { + if err := sstWriter.PutMVCCRangeKey(rangeKeys.AsRangeKey(v), mvccValue); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, err } } rows.BulkOpSummary.DataSize += rangeKeysSize - rangeKeys, rangeKeysEnd, rangeKeysSize = rangeKeys[:0], nil, 0 + rangeKeys, rangeKeysEnd, rangeKeysSize = MVCCRangeKeyStack{}, nil, 0 } // If we find any new range keys and we haven't buffered any range keys yet, // buffer them. - if hasRange && !skipTombstones && len(rangeKeys) == 0 { - rangeBounds := iter.RangeBounds() - rangeKeysEnd = append(rangeKeysEnd[:0], rangeBounds.EndKey...) - - for _, rkv := range iter.RangeKeys() { - rangeKeys = append(rangeKeys, rkv.Clone()) - rangeKeysSize += int64(len(rkv.RangeKey.StartKey) + len(rkv.RangeKey.EndKey) + len(rkv.Value)) - if !opts.ExportAllRevisions { - break - } + if hasRange && !skipTombstones && rangeKeys.IsEmpty() { + rangeKeys = iter.RangeKeys() + rangeKeysEnd = append(rangeKeysEnd[:0], rangeKeys.Bounds.EndKey...) + if !opts.ExportAllRevisions { + rangeKeys.Versions = rangeKeys.Versions[:1] + } + + // TODO(erikgrinaker): We should consider a CloneInto() method on the + // MVCCRangeKeyStack that allows reusing a byte buffer. See also TODO in + // Clone() about using a single allocation for the entire clone (all byte + // slices). + rangeKeys = rangeKeys.Clone() + + for _, v := range rangeKeys.Versions { + rangeKeysSize += int64( + len(rangeKeys.Bounds.Key) + len(rangeKeys.Bounds.EndKey) + len(v.Value)) } // Check if the range keys exceed a limit, using similar logic as point @@ -5571,10 +5565,10 @@ func MVCCExportToSST( // of some of the options and clean this up. curSize := rows.BulkOpSummary.DataSize reachedTargetSize := opts.TargetSize > 0 && uint64(curSize) >= opts.TargetSize - newSize := curSize + maxRangeKeysSizeIfTruncated(rangeBounds.Key) + newSize := curSize + maxRangeKeysSizeIfTruncated(rangeKeys.Bounds.Key) reachedMaxSize := opts.MaxSize > 0 && newSize > int64(opts.MaxSize) if paginated && (reachedTargetSize || reachedMaxSize) { - rangeKeys, rangeKeysEnd, rangeKeysSize = rangeKeys[:0], nil, 0 + rangeKeys, rangeKeysEnd, rangeKeysSize = MVCCRangeKeyStack{}, nil, 0 resumeKey = unsafeKey.Clone() break } @@ -5686,7 +5680,7 @@ func MVCCExportToSST( // next export's range keys to overlap with this one, e.g.: [a-f) with resume // key c@7 will export range keys [a-c\0) first, and then [c-f) when resuming, // which overlaps at [c-c\0). - if len(rangeKeys) > 0 { + if !rangeKeys.IsEmpty() { // Calculate the new rangeKeysSize due to the new resume bounds. if len(resumeKey.Key) > 0 && rangeKeysEnd.Compare(resumeKey.Key) > 0 { oldEndLen := len(rangeKeysEnd) @@ -5694,21 +5688,21 @@ func MVCCExportToSST( if resumeKey.Timestamp.IsSet() { rangeKeysEnd = rangeKeysEnd.Next() } - rangeKeysSize += int64(len(rangeKeys) * (len(rangeKeysEnd) - oldEndLen)) + rangeKeysSize += int64(rangeKeys.Len() * (len(rangeKeysEnd) - oldEndLen)) } - for _, rkv := range rangeKeys { - rkv.RangeKey.EndKey = rangeKeysEnd - mvccValue, ok, err := tryDecodeSimpleMVCCValue(rkv.Value) + rangeKeys.Bounds.EndKey = rangeKeysEnd + for _, v := range rangeKeys.Versions { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(v.Value) if !ok && err == nil { - mvccValue, err = decodeExtendedMVCCValue(rkv.Value) + mvccValue, err = decodeExtendedMVCCValue(v.Value) } if err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, - "decoding mvcc value %s", rkv.Value) + "decoding mvcc value %s", v.Value) } // Export only the inner roachpb.Value, not the MVCCValue header. mvccValue = MVCCValue{Value: mvccValue.Value} - if err := sstWriter.PutMVCCRangeKey(rkv.RangeKey, mvccValue); err != nil { + if err := sstWriter.PutMVCCRangeKey(rangeKeys.AsRangeKey(v), mvccValue); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, err } } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index bd753e7a3494..e1c5bff589e9 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -176,16 +176,17 @@ func TestMVCCHistories(t *testing.T) { break } hasData = true - buf.Printf("rangekey: %s/[", iter.RangeBounds()) - for i, rangeKV := range iter.RangeKeys() { - val, err := DecodeMVCCValue(rangeKV.Value) + rangeKeys := iter.RangeKeys() + buf.Printf("rangekey: %s/[", rangeKeys.Bounds) + for i, version := range rangeKeys.Versions { + val, err := DecodeMVCCValue(version.Value) if err != nil { t.Fatal(err) } if i > 0 { buf.Printf(" ") } - buf.Printf("%s=%s", rangeKV.RangeKey.Timestamp, val) + buf.Printf("%s=%s", version.Timestamp, val) } buf.Printf("]\n") iter.Next() @@ -1211,15 +1212,15 @@ func cmdExport(e *evalCtx) error { if rangeBounds := iter.RangeBounds(); !rangeBounds.Key.Equal(rangeStart) { rangeStart = append(rangeStart[:0], rangeBounds.Key...) e.results.buf.Printf("export: %s/[", rangeBounds) - for i, rangeKV := range iter.RangeKeys() { - val, err := DecodeMVCCValue(rangeKV.Value) + for i, version := range iter.RangeKeys().Versions { + val, err := DecodeMVCCValue(version.Value) if err != nil { return err } if i > 0 { e.results.buf.Printf(" ") } - e.results.buf.Printf("%s=%s", rangeKV.RangeKey.Timestamp, val) + e.results.buf.Printf("%s=%s", version.Timestamp, val) } e.results.buf.Printf("]\n") } @@ -1627,16 +1628,17 @@ func printIter(e *evalCtx) { } } if hasRange { - e.results.buf.Printf(" %s/[", e.iter.RangeBounds()) - for i, rangeKV := range e.iter.RangeKeys() { - value, err := DecodeMVCCValue(rangeKV.Value) + rangeKeys := e.iter.RangeKeys() + e.results.buf.Printf(" %s/[", rangeKeys.Bounds) + for i, version := range rangeKeys.Versions { + value, err := DecodeMVCCValue(version.Value) if err != nil { e.Fatalf("%v", err) } if i > 0 { e.results.buf.Printf(" ") } - e.results.buf.Printf("%s=%s", rangeKV.RangeKey.Timestamp, value) + e.results.buf.Printf("%s=%s", version.Timestamp, value) } e.results.buf.Printf("]") } diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 2b068ad09cc1..ac65b88a214d 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -11,8 +11,6 @@ package storage import ( - "sort" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" @@ -429,7 +427,9 @@ func (i *MVCCIncrementalIterator) advance() { } // NB: Don't update i.hasRange directly -- we only change it when - // i.rangeKeysStart changes, to avoid unnecessary checks. + // i.rangeKeysStart changes, which allows us to retain i.hasRange=false if + // we've already determined that the current range keys are outside of the + // time bounds. hasPoint, hasRange := i.iter.HasPointAndRange() i.hasPoint = hasPoint @@ -443,21 +443,10 @@ func (i *MVCCIncrementalIterator) advance() { if hasRange { if rangeStart := i.iter.RangeBounds().Key; !rangeStart.Equal(i.rangeKeysStart) { i.rangeKeysStart = append(i.rangeKeysStart[:0], rangeStart...) - // Find the first range key at or below EndTime. If that's also above - // StartTime then we have visible range keys. We use a linear search - // rather than a binary search because we expect EndTime to be near the - // current time, so the first range key will typically be sufficient. - hasRange = false - for _, rkv := range i.iter.RangeKeys() { - if ts := rkv.RangeKey.Timestamp; ts.LessEq(i.endTime) { - hasRange = i.startTime.Less(ts) - break - } - } - i.hasRange = hasRange - newRangeKey = hasRange + i.hasRange = i.iter.RangeKeys().HasBetween(i.startTime.Next(), i.endTime) + newRangeKey = i.hasRange } - // else keep i.hasRange from last i.rangeKeysStart change. + // Else: keep i.hasRange from last i.rangeKeysStart change. } else { i.hasRange = false } @@ -547,42 +536,18 @@ func (i *MVCCIncrementalIterator) RangeBounds() roachpb.Span { } // RangeKeys implements SimpleMVCCIterator. -func (i *MVCCIncrementalIterator) RangeKeys() []MVCCRangeKeyValue { +func (i *MVCCIncrementalIterator) RangeKeys() MVCCRangeKeyStack { if !i.hasRange { - return []MVCCRangeKeyValue{} + return MVCCRangeKeyStack{} } - // TODO(erikgrinaker): It may be worthwhile to clone and memoize this result // for the same range key. However, callers may avoid calling RangeKeys() // unnecessarily, and we may optimize parent iterators, so let's measure. rangeKeys := i.iter.RangeKeys() - if i.ignoringTime { return rangeKeys } - - // Find the first range key at or below endTime, and truncate rangeKeys. We do - // a linear search rather than a binary search, because we expect endTime to - // be near the current time, so the first element will typically match. - first := len(rangeKeys) - 1 - for idx, rkv := range rangeKeys { - if rkv.RangeKey.Timestamp.LessEq(i.endTime) { - first = idx - break - } - } - rangeKeys = rangeKeys[first:] - - // Find the first range key at or below startTime, and truncate rangeKeys. - if i.startTime.IsSet() { - if idx := sort.Search(len(rangeKeys), func(idx int) bool { - return rangeKeys[idx].RangeKey.Timestamp.LessEq(i.startTime) - }); idx >= 0 { - rangeKeys = rangeKeys[:idx] - } - } - - return rangeKeys + return rangeKeys.Trim(i.startTime.Next(), i.endTime) } // UnsafeValue implements SimpleMVCCIterator. diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 4791a6cb550c..587c9ad2aa6b 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -11,13 +11,13 @@ package storage import ( + "bytes" "encoding/binary" "fmt" "sort" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -365,6 +365,11 @@ type MVCCRangeKey struct { Timestamp hlc.Timestamp } +// Bounds returns the range key bounds as a Span. +func (k MVCCRangeKey) Bounds() roachpb.Span { + return roachpb.Span{Key: k.StartKey, EndKey: k.EndKey} +} + // Clone returns a copy of the range key. func (k MVCCRangeKey) Clone() MVCCRangeKey { // k is already a copy, but byte slices must be cloned. @@ -439,39 +444,228 @@ func (k MVCCRangeKey) Validate() (err error) { } } -// FirstRangeKeyAbove does a binary search for the first range key at or above -// the given timestamp. It assumes the range keys are ordered in descending -// timestamp order, as returned by SimpleMVCCIterator.RangeKeys(). Returns false -// if no matching range key was found. -// -// TODO(erikgrinaker): Consider using a new type for []MVCCRangeKeyValue as -// returned by SimpleMVCCIterator.RangeKeys(), and add this as a method. -func FirstRangeKeyAbove(rangeKeys []MVCCRangeKeyValue, ts hlc.Timestamp) (MVCCRangeKeyValue, bool) { - // This is kind of odd due to sort.Search() semantics: we do a binary search - // for the first range tombstone that's below the timestamp, then return the - // previous range tombstone if any. - if i := sort.Search(len(rangeKeys), func(i int) bool { - return rangeKeys[i].RangeKey.Timestamp.Less(ts) - }); i > 0 { - return rangeKeys[i-1], true +// MVCCRangeKeyStack represents a stack of range key fragments as returned +// by SimpleMVCCIterator.RangeKeys(). All fragments have the same key bounds, +// and are ordered from newest to oldest. +type MVCCRangeKeyStack struct { + Bounds roachpb.Span + Versions MVCCRangeKeyVersions +} + +// MVCCRangeKeyVersions represents a stack of range key fragment versions. +type MVCCRangeKeyVersions []MVCCRangeKeyVersion + +// MVCCRangeKeyVersion represents a single range key fragment version. +type MVCCRangeKeyVersion struct { + Timestamp hlc.Timestamp + Value []byte +} + +// AsRangeKey returns an MVCCRangeKey for the given version. Byte slices +// are shared with the stack. +func (s MVCCRangeKeyStack) AsRangeKey(v MVCCRangeKeyVersion) MVCCRangeKey { + return MVCCRangeKey{ + StartKey: s.Bounds.Key, + EndKey: s.Bounds.EndKey, + Timestamp: v.Timestamp, + } +} + +// AsRangeKeys converts the stack into a slice of MVCCRangeKey. Byte slices +// are shared with the stack. +func (s MVCCRangeKeyStack) AsRangeKeys() []MVCCRangeKey { + rangeKeys := make([]MVCCRangeKey, 0, len(s.Versions)) + for _, v := range s.Versions { + rangeKeys = append(rangeKeys, s.AsRangeKey(v)) + } + return rangeKeys +} + +// AsRangeKeyValue returns an MVCCRangeKeyValue for the given version. Byte +// slices are shared with the stack. +func (s MVCCRangeKeyStack) AsRangeKeyValue(v MVCCRangeKeyVersion) MVCCRangeKeyValue { + return MVCCRangeKeyValue{ + RangeKey: s.AsRangeKey(v), + Value: v.Value, + } +} + +// AsRangeKeyValues converts the stack into a slice of MVCCRangeKeyValue. Byte +// slices are shared with the stack. +func (s MVCCRangeKeyStack) AsRangeKeyValues() []MVCCRangeKeyValue { + kvs := make([]MVCCRangeKeyValue, 0, len(s.Versions)) + for _, v := range s.Versions { + kvs = append(kvs, s.AsRangeKeyValue(v)) } - return MVCCRangeKeyValue{}, false + return kvs } -// HasRangeKeyBetween checks whether an MVCC range key exists between the two -// given timestamps (in order). It assumes the range keys are ordered in -// descending timestamp order, as returned by SimpleMVCCIterator.RangeKeys(). -func HasRangeKeyBetween(rangeKeys []MVCCRangeKeyValue, upper, lower hlc.Timestamp) bool { - if len(rangeKeys) == 0 { +// CanMergeRight returns true if the current stack will merge with the given +// right-hand stack. The key bounds must touch exactly, i.e. the left-hand +// EndKey must equal the right-hand Key. +func (s MVCCRangeKeyStack) CanMergeRight(r MVCCRangeKeyStack) bool { + if s.IsEmpty() || s.Len() != r.Len() || !s.Bounds.EndKey.Equal(r.Bounds.Key) { return false } - if util.RaceEnabled && upper.Less(lower) { - panic(errors.AssertionFailedf("HasRangeKeyBetween given upper %s <= lower %s", upper, lower)) + for i := range s.Versions { + if !s.Versions[i].Equal(r.Versions[i]) { + return false + } } - if rkv, ok := FirstRangeKeyAbove(rangeKeys, lower); ok { + return true +} + +// Clone clones the stack. +func (s MVCCRangeKeyStack) Clone() MVCCRangeKeyStack { + // TODO(erikgrinaker): We can optimize this by using a single memory + // allocation for all byte slices in the entire stack. + s.Bounds = s.Bounds.Clone() + s.Versions = s.Versions.Clone() + return s +} + +// Covers returns true if any range key in the stack covers the given point key. +func (s MVCCRangeKeyStack) Covers(k MVCCKey) bool { + return s.Versions.Covers(k.Timestamp) && s.Bounds.ContainsKey(k.Key) +} + +// CoversTimestamp returns true if any range key in the stack covers the given timestamp. +func (s MVCCRangeKeyStack) CoversTimestamp(ts hlc.Timestamp) bool { + return s.Versions.Covers(ts) +} + +// FirstAbove does a binary search for the first range key version at or above +// the given timestamp. Returns false if no matching range key was found. +func (s MVCCRangeKeyStack) FirstAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + return s.Versions.FirstAbove(ts) +} + +// FirstBelow does a binary search for the first range key version at or below +// the given timestamp. Returns false if no matching range key was found. +func (s MVCCRangeKeyStack) FirstBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + return s.Versions.FirstBelow(ts) +} + +// HasBetween checks whether an MVCC range key exists between the two given +// timestamps (both inclusive, in order). +func (s MVCCRangeKeyStack) HasBetween(lower, upper hlc.Timestamp) bool { + return s.Versions.HasBetween(lower, upper) +} + +// IsEmpty returns true if the stack is empty (no versions). +func (s MVCCRangeKeyStack) IsEmpty() bool { + return s.Versions.IsEmpty() +} + +// Len returns the number of versions in the stack. +func (s MVCCRangeKeyStack) Len() int { + return len(s.Versions) +} + +// Timestamps returns the timestamps of all versions. +func (s MVCCRangeKeyStack) Timestamps() []hlc.Timestamp { + return s.Versions.Timestamps() +} + +// Trim trims the versions to the time span [from, to] (both inclusive). +func (s MVCCRangeKeyStack) Trim(from, to hlc.Timestamp) MVCCRangeKeyStack { + s.Versions = s.Versions.Trim(from, to) + return s +} + +// Clone clones the versions. +func (v MVCCRangeKeyVersions) Clone() MVCCRangeKeyVersions { + c := make(MVCCRangeKeyVersions, len(v)) + for i, version := range v { + c[i] = version.Clone() + } + return c +} + +// Covers returns true if any version in the stack is above the given timestamp. +func (v MVCCRangeKeyVersions) Covers(ts hlc.Timestamp) bool { + return !v.IsEmpty() && ts.LessEq(v[0].Timestamp) +} + +// FirstAbove does a binary search for the first range key version at or above +// the given timestamp. Returns false if no matching range key was found. +func (v MVCCRangeKeyVersions) FirstAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + // This is kind of odd due to sort.Search() semantics: we do a binary search + // for the first range key that's below the timestamp, then return the + // previous range key if any. + if i := sort.Search(len(v), func(i int) bool { + return v[i].Timestamp.Less(ts) + }); i > 0 { + return v[i-1], true + } + return MVCCRangeKeyVersion{}, false +} + +// FirstBelow does a binary search for the first range key version at or below +// the given timestamp. Returns false if no matching range key was found. +func (v MVCCRangeKeyVersions) FirstBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + if i := sort.Search(len(v), func(i int) bool { + return v[i].Timestamp.LessEq(ts) + }); i < len(v) { + return v[i], true + } + return MVCCRangeKeyVersion{}, false +} + +// HasBetween checks whether an MVCC range key exists between the two given +// timestamps (both inclusive, in order). +func (v MVCCRangeKeyVersions) HasBetween(lower, upper hlc.Timestamp) bool { + if version, ok := v.FirstAbove(lower); ok { // Consider equal timestamps to be "between". This shouldn't really happen, // since MVCC enforces point and range keys can't have the same timestamp. - return rkv.RangeKey.Timestamp.LessEq(upper) + return version.Timestamp.LessEq(upper) } return false } + +// IsEmpty returns true if the stack is empty (no versions). +func (v MVCCRangeKeyVersions) IsEmpty() bool { + return len(v) == 0 +} + +// Timestamps returns the timestamps of all versions. +func (v MVCCRangeKeyVersions) Timestamps() []hlc.Timestamp { + timestamps := make([]hlc.Timestamp, 0, len(v)) + for _, version := range v { + timestamps = append(timestamps, version.Timestamp) + } + return timestamps +} + +// Trim trims the versions to the time span [from, to] (both inclusive). +func (v MVCCRangeKeyVersions) Trim(from, to hlc.Timestamp) MVCCRangeKeyVersions { + // We assume that to will often be near the current time, and use a linear + // rather than a binary search, which will often match on the first range key. + start := len(v) + for i, version := range v { + if version.Timestamp.LessEq(to) { + start = i + break + } + } + v = v[start:] + + // We then use a binary search to find the lower bound. + end := sort.Search(len(v), func(i int) bool { + return v[i].Timestamp.Less(from) + }) + return v[:end] +} + +// Clone clones the version. +func (v MVCCRangeKeyVersion) Clone() MVCCRangeKeyVersion { + if v.Value != nil { + v.Value = append([]byte(nil), v.Value...) + } + return v +} + +// Equal returns true if the two versions are equal. +func (v MVCCRangeKeyVersion) Equal(o MVCCRangeKeyVersion) bool { + return v.Timestamp.Equal(o.Timestamp) && bytes.Equal(v.Value, o.Value) +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index e0525d146605..a3e8c33cd3eb 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -22,7 +22,6 @@ import ( "testing/quick" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -499,19 +498,88 @@ func TestMVCCRangeKeyValidate(t *testing.T) { } } -func TestFirstRangeKeyAbove(t *testing.T) { +func TestMVCCRangeKeyStackCanMergeRight(t *testing.T) { defer leaktest.AfterTest(t)() - rangeKVs := []MVCCRangeKeyValue{ - rangeKV("a", "f", 6, MVCCValue{}), - rangeKV("a", "f", 4, MVCCValue{}), - rangeKV("a", "f", 3, MVCCValue{}), - rangeKV("a", "f", 1, MVCCValue{}), + testcases := map[string]struct { + lhs, rhs MVCCRangeKeyStack + expect bool + }{ + "empty stacks don't merge": { + rangeKeyStack("", "", nil), + rangeKeyStack("", "", nil), + false}, + + "empty lhs doesn't merge": { + rangeKeyStack("a", "b", map[int]MVCCValue{}), + rangeKeyStack("b", "c", map[int]MVCCValue{1: {}}), + false}, + + "empty rhs doesn't merge": { + rangeKeyStack("a", "b", map[int]MVCCValue{1: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{}), + false}, + + "stacks merge": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + true}, + + "missing lhs version end": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + + "missing rhs version end": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 3: {}}), + false}, + + "different version timestamp": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 2: {}, 1: {}}), + false}, + + "different version value": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: tombstoneLocalTS(9), 1: {}}), + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + + "bounds not touching": { + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("c", "d", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + + "overlapping range keys don't merge": { + rangeKeyStack("a", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("b", "d", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + + "same range keys don't merge": { + rangeKeyStack("a", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("a", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + + "wrong order don't merge": { + rangeKeyStack("b", "c", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + rangeKeyStack("a", "b", map[int]MVCCValue{5: {}, 3: {}, 1: {}}), + false}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + require.Equal(t, tc.expect, tc.lhs.CanMergeRight(tc.rhs)) + }) } +} + +func TestMVCCRangeKeyStackFirstAbove(t *testing.T) { + defer leaktest.AfterTest(t)() + + rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{6: {}, 4: {}, 3: {}, 1: {}}) testcases := []struct { - ts int64 - expect int64 + ts int + expect int }{ {0, 1}, {1, 1}, @@ -524,51 +592,109 @@ func TestFirstRangeKeyAbove(t *testing.T) { } for _, tc := range testcases { t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { - rkv, ok := FirstRangeKeyAbove(rangeKVs, hlc.Timestamp{WallTime: tc.ts}) + v, ok := rangeKeys.FirstAbove(wallTS(tc.ts)) if tc.expect == 0 { require.False(t, ok) - require.Empty(t, rkv) + require.Empty(t, v) } else { require.True(t, ok) - require.Equal(t, rangeKV("a", "f", int(tc.expect), MVCCValue{}), rkv) + require.Equal(t, rangeKeyVersion(tc.expect, MVCCValue{}), v) } }) } } -func TestHasRangeKeyBetween(t *testing.T) { +func TestMVCCRangeKeyStackFirstBelow(t *testing.T) { defer leaktest.AfterTest(t)() - rangeKVs := []MVCCRangeKeyValue{ - rangeKV("a", "f", 5, MVCCValue{}), - rangeKV("a", "f", 1, MVCCValue{}), + rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{6: {}, 4: {}, 3: {}, 1: {}}) + + testcases := []struct { + ts int + expect int + }{ + {0, 0}, + {1, 1}, + {2, 1}, + {3, 3}, + {4, 4}, + {5, 4}, + {6, 6}, + {7, 6}, } + for _, tc := range testcases { + t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { + v, ok := rangeKeys.FirstBelow(wallTS(tc.ts)) + if tc.expect == 0 { + require.False(t, ok) + require.Empty(t, v) + } else { + require.True(t, ok) + require.Equal(t, rangeKeyVersion(tc.expect, MVCCValue{}), v) + } + }) + } +} + +func TestMVCCRangeKeyStackHasBetween(t *testing.T) { + defer leaktest.AfterTest(t)() + + rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{5: {}, 1: {}}) testcases := []struct { - upper, lower int + lower, upper int expect bool }{ {0, 0, false}, - {0, 1, false}, // wrong order - {1, 0, true}, + {1, 0, false}, // wrong order + {0, 1, true}, {1, 1, true}, - {0, 2, false}, // wrong order - {4, 6, false}, // wrong order - {6, 4, true}, + {2, 0, false}, // wrong order + {6, 4, false}, // wrong order + {4, 6, true}, {5, 5, true}, {4, 4, false}, {6, 6, false}, - {4, 2, false}, - {0, 9, false}, // wrong order - {9, 0, true}, + {2, 4, false}, + {9, 0, false}, // wrong order + {0, 9, true}, } for _, tc := range testcases { - t.Run(fmt.Sprintf("%d,%d", tc.upper, tc.lower), func(t *testing.T) { - if util.RaceEnabled && tc.upper < tc.lower { - require.Panics(t, func() { HasRangeKeyBetween(rangeKVs, wallTS(tc.upper), wallTS(tc.lower)) }) - } else { - require.Equal(t, tc.expect, HasRangeKeyBetween(rangeKVs, wallTS(tc.upper), wallTS(tc.lower))) + t.Run(fmt.Sprintf("%d,%d", tc.lower, tc.upper), func(t *testing.T) { + require.Equal(t, tc.expect, rangeKeys.HasBetween(wallTS(tc.lower), wallTS(tc.upper))) + }) + } +} + +func TestMVCCRangeKeyStackTrim(t *testing.T) { + defer leaktest.AfterTest(t)() + + rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{7: {}, 5: {}, 3: {}}) + + testcases := []struct { + from, to int + expect []int + }{ + {0, 10, []int{7, 5, 3}}, + {10, 0, []int{}}, // wrong order + {0, 0, []int{}}, + {0, 2, []int{}}, + {8, 9, []int{}}, + {3, 7, []int{7, 5, 3}}, + {4, 7, []int{7, 5}}, + {4, 6, []int{5}}, + {5, 6, []int{5}}, + {4, 5, []int{5}}, + {5, 5, []int{5}}, + } + for _, tc := range testcases { + t.Run(fmt.Sprintf("%d,%d", tc.from, tc.to), func(t *testing.T) { + expect := rangeKeyStack("a", "f", nil) + for _, ts := range tc.expect { + expect.Versions = append(expect.Versions, rangeKeyVersion(ts, MVCCValue{})) } + + require.Equal(t, expect, rangeKeys.Trim(wallTS(tc.from), wallTS(tc.to))) }) } } @@ -608,6 +734,37 @@ func rangeKV(start, end string, ts int, v MVCCValue) MVCCRangeKeyValue { } } +func rangeKeyStack(start, end string, versions map[int]MVCCValue) MVCCRangeKeyStack { + return MVCCRangeKeyStack{ + Bounds: roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)}, + Versions: rangeKeyVersions(versions), + } +} + +func rangeKeyVersions(v map[int]MVCCValue) MVCCRangeKeyVersions { + versions := make([]MVCCRangeKeyVersion, len(v)) + var timestamps []int + for i := range v { + timestamps = append(timestamps, i) + } + sort.Ints(timestamps) + for i, ts := range timestamps { + versions[len(versions)-1-i] = rangeKeyVersion(ts, v[ts]) + } + return versions +} + +func rangeKeyVersion(ts int, v MVCCValue) MVCCRangeKeyVersion { + valueRaw, err := EncodeMVCCValue(v) + if err != nil { + panic(err) + } + return MVCCRangeKeyVersion{ + Timestamp: wallTS(ts), + Value: valueRaw, + } +} + func wallTS(ts int) hlc.Timestamp { return hlc.Timestamp{WallTime: int64(ts)} } diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 59900718a7f7..3e4d8d0e3d31 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -5763,9 +5763,9 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { if !ok { break } - for _, rkv := range it.RangeKeys() { - require.Less(t, expectIndex, len(d.after), "not enough expectations; at unexpected range:", rkv.RangeKey.String()) - require.EqualValues(t, d.after[expectIndex], rkv.RangeKey, "range key is not equal") + for _, rk := range it.RangeKeys().AsRangeKeys() { + require.Less(t, expectIndex, len(d.after), "not enough expectations; at unexpected range: %s", rk) + require.EqualValues(t, d.after[expectIndex], rk, "range key is not equal") expectIndex++ } } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index eeeddc7bd769..da919c907474 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -663,10 +663,12 @@ func (p *pebbleIterator) EngineRangeBounds() (roachpb.Span, error) { } // RangeKeys implements the MVCCIterator interface. -func (p *pebbleIterator) RangeKeys() []MVCCRangeKeyValue { - bounds := p.RangeBounds() +func (p *pebbleIterator) RangeKeys() MVCCRangeKeyStack { rangeKeys := p.iter.RangeKeys() - rangeKVs := make([]MVCCRangeKeyValue, 0, len(rangeKeys)) + stack := MVCCRangeKeyStack{ + Bounds: p.RangeBounds(), + Versions: make(MVCCRangeKeyVersions, 0, len(rangeKeys)), + } for _, rangeKey := range rangeKeys { timestamp, err := DecodeMVCCTimestampSuffix(rangeKey.Suffix) @@ -675,16 +677,12 @@ func (p *pebbleIterator) RangeKeys() []MVCCRangeKeyValue { // we follow UnsafeKey()'s example and silently skip them. continue } - rangeKVs = append(rangeKVs, MVCCRangeKeyValue{ - RangeKey: MVCCRangeKey{ - StartKey: bounds.Key, - EndKey: bounds.EndKey, - Timestamp: timestamp, - }, - Value: rangeKey.Value, + stack.Versions = append(stack.Versions, MVCCRangeKeyVersion{ + Timestamp: timestamp, + Value: rangeKey.Value, }) } - return rangeKVs + return stack } // EngineRangeKeys implements the EngineIterator interface. diff --git a/pkg/storage/point_synthesizing_iter.go b/pkg/storage/point_synthesizing_iter.go index 53947213995c..2f4ecd26aab3 100644 --- a/pkg/storage/point_synthesizing_iter.go +++ b/pkg/storage/point_synthesizing_iter.go @@ -65,9 +65,9 @@ var pointSynthesizingIterPool = sync.Pool{ type pointSynthesizingIter struct { iter MVCCIterator - // rangeKeys contains any range keys that overlap the current key position, - // for which points will be synthesized. - rangeKeys []MVCCRangeKeyValue + // rangeKeys contains any range key versions that overlap the current key + // position, for which points will be synthesized. + rangeKeys MVCCRangeKeyVersions // rangeKeysPos is the current key (along the rangeKeys span) that points will // be synthesized for. It is only set if rangeKeys is non-empty, and may @@ -172,14 +172,11 @@ func (i *pointSynthesizingIter) updateRangeKeys() { if !i.iterValid { i.clearRangeKeys() } else if _, hasRange := i.iter.HasPointAndRange(); hasRange { + // TODO(erikgrinaker): Optimize this. i.rangeKeysPos = append(i.rangeKeysPos[:0], i.iter.UnsafeKey().Key...) if rangeStart := i.iter.RangeBounds().Key; !rangeStart.Equal(i.rangeKeysStart) { i.rangeKeysStart = append(i.rangeKeysStart[:0], rangeStart...) - i.rangeKeys = i.rangeKeys[:0] - for _, rk := range i.iter.RangeKeys() { - // TODO(erikgrinaker): We should optimize the clone cost. - i.rangeKeys = append(i.rangeKeys, rk.Clone()) - } + i.rangeKeys = i.iter.RangeKeys().Versions.Clone() } if !i.reverse { i.rangeKeysIdx = 0 @@ -206,10 +203,10 @@ func (i *pointSynthesizingIter) updateAtPoint() { } else if !i.reverse { i.atPoint = i.rangeKeysIdx >= len(i.rangeKeys) || !point.Timestamp.IsSet() || - i.rangeKeys[i.rangeKeysIdx].RangeKey.Timestamp.LessEq(point.Timestamp) + i.rangeKeys[i.rangeKeysIdx].Timestamp.LessEq(point.Timestamp) } else { i.atPoint = i.rangeKeysIdx < 0 || (point.Timestamp.IsSet() && - point.Timestamp.LessEq(i.rangeKeys[i.rangeKeysIdx].RangeKey.Timestamp)) + point.Timestamp.LessEq(i.rangeKeys[i.rangeKeysIdx].Timestamp)) } } @@ -341,7 +338,7 @@ func (i *pointSynthesizingIter) SeekGE(seekKey MVCCKey) { // If we're seeking to a specific version, skip newer range keys. if len(i.rangeKeys) > 0 && seekKey.Timestamp.IsSet() && seekKey.Key.Equal(i.rangeKeysPos) { i.rangeKeysIdx = sort.Search(len(i.rangeKeys), func(idx int) bool { - return i.rangeKeys[idx].RangeKey.Timestamp.LessEq(seekKey.Timestamp) + return i.rangeKeys[idx].Timestamp.LessEq(seekKey.Timestamp) }) } @@ -492,7 +489,7 @@ func (i *pointSynthesizingIter) SeekLT(seekKey MVCCKey) { // If we're seeking to a specific version, skip over older range keys. if seekKey.Timestamp.IsSet() && seekKey.Key.Equal(i.rangeKeysPos) { i.rangeKeysIdx = sort.Search(len(i.rangeKeys), func(idx int) bool { - return i.rangeKeys[idx].RangeKey.Timestamp.LessEq(seekKey.Timestamp) + return i.rangeKeys[idx].Timestamp.LessEq(seekKey.Timestamp) }) - 1 } @@ -569,7 +566,7 @@ func (i *pointSynthesizingIter) UnsafeKey() MVCCKey { } return MVCCKey{ Key: i.rangeKeysPos, - Timestamp: i.rangeKeys[i.rangeKeysIdx].RangeKey.Timestamp, + Timestamp: i.rangeKeys[i.rangeKeysIdx].Timestamp, } } @@ -624,8 +621,8 @@ func (i *pointSynthesizingIter) RangeBounds() roachpb.Span { } // RangeKeys implements MVCCIterator. -func (i *pointSynthesizingIter) RangeKeys() []MVCCRangeKeyValue { - return []MVCCRangeKeyValue{} +func (i *pointSynthesizingIter) RangeKeys() MVCCRangeKeyStack { + return MVCCRangeKeyStack{} } // ComputeStats implements MVCCIterator. @@ -752,10 +749,10 @@ func (i *pointSynthesizingIter) assertInvariants() error { maxIdx = i.rangeKeysIdx + 1 } if minIdx >= 0 && minIdx < len(i.rangeKeys) { - minKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[minIdx].RangeKey.Timestamp} + minKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[minIdx].Timestamp} } if maxIdx >= 0 && maxIdx < len(i.rangeKeys) { - maxKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[maxIdx].RangeKey.Timestamp} + maxKey = MVCCKey{Key: i.rangeKeysPos, Timestamp: i.rangeKeys[maxIdx].Timestamp} } iterKey := i.iter.Key() diff --git a/pkg/storage/read_as_of_iterator.go b/pkg/storage/read_as_of_iterator.go index 65afef5910b0..a88862dd9e93 100644 --- a/pkg/storage/read_as_of_iterator.go +++ b/pkg/storage/read_as_of_iterator.go @@ -110,8 +110,8 @@ func (f *ReadAsOfIterator) RangeBounds() roachpb.Span { } // RangeKeys is always empty since this iterator never surfaces rangeKeys. -func (f *ReadAsOfIterator) RangeKeys() []MVCCRangeKeyValue { - return []MVCCRangeKeyValue{} +func (f *ReadAsOfIterator) RangeKeys() MVCCRangeKeyStack { + return MVCCRangeKeyStack{} } // updateValid updates i.valid and i.err based on the underlying iterator, and @@ -130,7 +130,9 @@ func (f *ReadAsOfIterator) advance() { return } - if !f.asOf.IsEmpty() && f.asOf.Less(f.iter.UnsafeKey().Timestamp) { + key := f.iter.UnsafeKey() + + if f.asOf.Less(key.Timestamp) { // Skip keys above the asOf timestamp, regardless of type of key (e.g. point or range) f.iter.Next() } else if hasPoint, hasRange := f.iter.HasPointAndRange(); !hasPoint && hasRange { @@ -155,31 +157,25 @@ func (f *ReadAsOfIterator) advance() { } else if !hasRange { // On a valid key without a range key return - } else if f.asOfRangeKeyShadows() { + // TODO (msbutler): ensure this caches range key values (#84379) before + // the 22.2 branch cut, else we face a steep perf cliff for RESTORE with + // range keys. + } else if f.iter.RangeKeys().HasBetween(key.Timestamp, f.asOf) { // The latest range key, as of system time, shadows the latest point key. // This key is therefore deleted as of system time. f.iter.NextKey() } else { - // On a valid key that potentially shadows range key(s) + // On a valid key that potentially shadows range key(s). return } } } -// asOfRangeKeyShadows returns true if there exists a range key at or below the asOf timestamp -// that shadows the latest point key -// -// TODO (msbutler): ensure this function caches range key values (#84379) before -// the 22.2 branch cut, else we face a steep perf cliff for RESTORE with range keys. -func (f *ReadAsOfIterator) asOfRangeKeyShadows() (shadows bool) { - rangeKeys := f.iter.RangeKeys() - if f.asOf.IsEmpty() { - return f.iter.UnsafeKey().Timestamp.LessEq(rangeKeys[0].RangeKey.Timestamp) - } - return HasRangeKeyBetween(rangeKeys, f.asOf, f.iter.UnsafeKey().Timestamp) -} - -// NewReadAsOfIterator constructs a ReadAsOfIterator. +// NewReadAsOfIterator constructs a ReadAsOfIterator. If asOf is not set, the +// iterator reads the most recent data. func NewReadAsOfIterator(iter SimpleMVCCIterator, asOf hlc.Timestamp) *ReadAsOfIterator { + if asOf.IsEmpty() { + asOf = hlc.MaxTimestamp + } return &ReadAsOfIterator{iter: iter, asOf: asOf} } diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index f0ad9c1c601e..ef004407f158 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -409,13 +409,14 @@ func UpdateSSTTimestamps( } else if !ok { break } - for _, rkv := range iter.RangeKeys() { - if rkv.RangeKey.Timestamp != from { + rangeKeys := iter.RangeKeys() + for _, v := range rangeKeys.Versions { + if v.Timestamp != from { return nil, errors.Errorf("unexpected timestamp %s (expected %s) for range key %s", - rkv.RangeKey.Timestamp, from, rkv.RangeKey) + v.Timestamp, from, rangeKeys.Bounds) } - rkv.RangeKey.Timestamp = to - if err = writer.PutRawMVCCRangeKey(rkv.RangeKey, rkv.Value); err != nil { + v.Timestamp = to + if err = writer.PutRawMVCCRangeKey(rangeKeys.AsRangeKey(v), v.Value); err != nil { return nil, err } } diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index 2e5269111b3e..72c3e302278e 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -232,6 +232,6 @@ func (r *sstIterator) RangeBounds() roachpb.Span { } // RangeKeys implements SimpleMVCCIterator. -func (r *sstIterator) RangeKeys() []MVCCRangeKeyValue { - return []MVCCRangeKeyValue{} +func (r *sstIterator) RangeKeys() MVCCRangeKeyStack { + return MVCCRangeKeyStack{} } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index cce9a0daf8a6..9d5b65366d80 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -2208,6 +2208,8 @@ func TestLint(t *testing.T) { stream.GrepNot(`pkg/roachprod/logger/log\.go:.*format argument is not a constant expression`), // We purposefully produce nil dereferences in this file to test crash conditions stream.GrepNot(`pkg/util/log/logcrash/crash_reporting_test\.go:.*nil dereference in type assertion`), + // Temporarily copied code from google-cloud-go's retry predicate. + stream.GrepNot(`pkg/cloud/gcp/gcs_retry\.go:.*invalid direct cast on error object`), // Spawning naked goroutines is ok when it's not as part of the main CRDB // binary. This is for now - if we use #58164 to introduce more aggressive // pooling, etc, then test code needs to adhere as well. diff --git a/pkg/testutils/storageutils/scan.go b/pkg/testutils/storageutils/scan.go index ac22f74613df..bd21e5377ef6 100644 --- a/pkg/testutils/storageutils/scan.go +++ b/pkg/testutils/storageutils/scan.go @@ -45,13 +45,14 @@ func ScanIter(t *testing.T, iter storage.SimpleMVCCIterator) KVs { hasPoint, hasRange := iter.HasPointAndRange() if hasRange { if bounds := iter.RangeBounds(); !bounds.Key.Equal(prevRangeStart) { - for _, rkv := range iter.RangeKeys() { - if len(rkv.Value) == 0 { - rkv.Value = nil + prevRangeStart = bounds.Key.Clone() + rangeKeys := iter.RangeKeys().Clone() + for _, v := range rangeKeys.Versions { + if len(v.Value) == 0 { + v.Value = nil } - kvs = append(kvs, rkv.Clone()) + kvs = append(kvs, rangeKeys.AsRangeKeyValue(v)) } - prevRangeStart = bounds.Key.Clone() } } if hasPoint { diff --git a/pkg/util/log/eventpb/gen.go b/pkg/util/log/eventpb/gen.go index 9a42e493c7a4..c58bed91f5b1 100644 --- a/pkg/util/log/eventpb/gen.go +++ b/pkg/util/log/eventpb/gen.go @@ -343,6 +343,11 @@ func readInput( return errors.Newf("field definition must not span multiple lines: %q", line) } + // Skip reserved fields. + if reservedDefRe.MatchString(line) { + continue + } + // A field. if strings.HasPrefix(line, "repeated") { line = "array_of_" + strings.TrimSpace(strings.TrimPrefix(line, "repeated")) @@ -448,6 +453,8 @@ var fieldDefRe = regexp.MustCompile(`\s*(?P[a-z._A-Z0-9]+)` + `(.*"redact:\\"safeif:(?P([^\\]|\\[^"])+)\\"")?` + `).*$`) +var reservedDefRe = regexp.MustCompile(`\s*(reserved ([1-9][0-9]*);)`) + func camelToSnake(typeName string) string { var res strings.Builder res.WriteByte(typeName[0] + 'a' - 'A') diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index e0d135ed3c3f..7090cb1e274d 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -3335,15 +3335,6 @@ func (m *SampledQuery) AppendJSONFields(printComma bool, b redact.RedactableByte b = append(b, '"') } - if m.DatabaseID != 0 { - if printComma { - b = append(b, ',') - } - printComma = true - b = append(b, "\"DatabaseID\":"...) - b = strconv.AppendUint(b, uint64(m.DatabaseID), 10) - } - if m.StatementFingerprintID != 0 { if printComma { b = append(b, ',') diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index 80b24562a960..4f575bd51f5b 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -59,9 +59,6 @@ message SampledQuery { // Transaction ID of the query. string transaction_id = 11 [(gogoproto.customname) = "TransactionID", (gogoproto.jsontag) = ',omitempty', (gogoproto.moretags) = "redact:\"nonsensitive\""]; - // Database ID of the query. - uint32 database_id = 12 [(gogoproto.customname) = "DatabaseID", (gogoproto.jsontag) = ",omitempty"]; - // Statement fingerprint ID of the query. uint64 statement_fingerprint_id = 13 [(gogoproto.customname) = "StatementFingerprintID", (gogoproto.jsontag) = ',omitempty']; @@ -92,6 +89,8 @@ message SampledQuery { // The number of rows written. int64 rows_written = 21 [(gogoproto.jsontag) = ",omitempty"]; + + reserved 12; } // CapturedIndexUsageStats