From 00362eb19441d653d2ba1ed8bff89fd93847bd21 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 4 Jan 2023 19:48:18 -0800 Subject: [PATCH] sql,storage: add support for COL_BATCH_RESPONSE scan format MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit introduces a new `COL_BATCH_RESPONSE` scan format for Scans and ReverseScans which results only in needed columns to be returned from the KV server. In other words, this commit introduces the ability to perform the KV projection pushdown. The main idea of this feature is to use the injected decoding logic from SQL in order to process each KV and keep only the needed parts (i.e. necessary SQL columns). Those needed parts are then propagated back to the KV client as coldata.Batch'es (serialized in the Apache Arrow format). Here is the outline of all components involved: ┌────────────────────────────────────────────────┐ │ SQL │ │________________________________________________│ │ colfetcher.ColBatchDirectScan │ │ │ │ │ ▼ │ │ row.txnKVFetcher │ │ (behind the row.KVBatchFetcher interface) │ └────────────────────────────────────────────────┘ │ ▼ ┌────────────────────────────────────────────────┐ │ KV Client │ └────────────────────────────────────────────────┘ │ ▼ ┌────────────────────────────────────────────────┐ │ KV Server │ │________________________________________________│ │ colfetcher.cFetcherWrapper │ │ (behind the storage.CFetcherWrapper interface) │ │ │ │ │ ▼ │ │ colfetcher.cFetcher │ │ │ │ │ ▼ │ │ storage.mvccScanFetchAdapter ────────┐│ │ (behind the storage.NextKVer interface) ││ │ │ ││ │ ▼ ││ │ storage.pebbleMVCCScanner ││ │ (which put's KVs into storage.singleResults) <┘│ └────────────────────────────────────────────────┘ On the KV client side, `row.txnKVFetcher` issues Scans and ReverseScans with the `COL_BATCH_RESPONSE` format and returns the response (which contains the columnar data) to the `colfetcher.ColBatchDirectScan`. On the KV server side, we create a `storage.CFetcherWrapper` that asks the `colfetcher.cFetcher` for the next `coldata.Batch`. The `cFetcher`, in turn, fetches the next KV, decodes it, and keeps only values for the needed SQL columns, discarding the rest of the KV. The KV is emitted by the `mvccScanFetchAdapter` which - via the `singleResults` struct - exposes access to the current KV that the `pebbleMVCCScanner` is pointing at. Note that there is an additional "implicit synchronization" between components that is not shown on this diagram. In particular, `storage.singleResults.maybeTrimPartialLastRow` must be in sync with the `colfetcher.cFetcher` which is achieved by - the `cFetcher` exposing access to the first key of the last incomplete SQL row via the `FirstKeyOfRowGetter`, - the `singleResults` using that key as the resume key for the response, - and the `cFetcher` removing that last partial SQL row when `NextKV()` returns `partialRow=true`. This "upstream" link (although breaking the layering a bit) allows us to avoid a performance penalty for handling the case with multiple column families. (This case is handled by the `storage.pebbleResults` via tracking offsets into the `pebbleResults.repr`.) This code structure deserves some elaboration. First, there is a mismatch between the "push" mode in which the `pebbleMVCCScanner` operates and the "pull" mode that the `NextKVer` exposes. The adaption between two different modes is achieved via the `mvccScanFetcherAdapter` grabbing (when the control returns to it) the current unstable KV pair from the `singleResults` struct which serves as a one KV pair buffer that the `pebbleMVCCScanner` `put`s into. Second, in order be able to use the unstable KV pair without performing a copy, the `pebbleMVCCScanner` stops at the current KV pair and returns the control flow (which is exactly what `pebbleMVCCScanner.getOne` does) back to the `mvccScanFetcherAdapter`, with the adapter advancing the scanner only when the next KV pair is needed. There are multiple scenarios which are currently not supported: - SQL cannot issue Get requests (likely will support in 23.1) - `TraceKV` option is not supported (likely will support in 23.1) - user-defined types other than enums are not supported (will _not_ support in 23.1) - non-default key locking strength as well as SKIP LOCKED wait policy are not supported (will _not_ support in 23.1). The usage of this feature is currently disabled by default, but I intend to enable it by default for multi-tenant setups. The rationale is that currently there is a large performance hit when enabling it for single-tenant deployments whereas it offers significant speed up in the multi-tenant world. TODO: benchmark number for multi-tenant setup. At the moment, `coldata.Batch` that is included into the response is always serialized into the Arrow format, but I intend to introduce the local fastpath to avoid that serialization. That work will be done in a follow-up and should be able to reduce the perf hit for single-tenant deployments. A quick note on the TODOs sprinkled in this commit: - `TODO(yuzefovich)` means that this will be left for 23.2 or later. - `TODO(yuzefovich, 23.1)` means that it should be addressed in 23.1. - `TODO(feedback wanted)` is something that I don't have a strong opinion on, so feedback from everyone is very welcome. A quick note on testing: this commit randomizes the fact whether the new infrastructure is used in almost all test builds. Introducing some unit testing (say, in `storage` package) seems rather annoying since we must create keys that are valid SQL keys (i.e. have TableID / Index ID prefix) and need to come with the corresponding `fetchpb.IndexFetchSpec`. Not having unit tests in the `storage` seems ok to me given that the "meat" of the work there is still done by the `pebbleMVCCScanner` which is exercised using the regular Scans. End-to-end testing is well covered by all of our existing tests which now runs randomly. I did run the CI multiple times with the new feature enabled by default with no failure, so I hope that it shouldn't become flaky. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- .../generated_test.go | 3 + .../tests/3node-tenant/generated_test.go | 3 + pkg/clusterversion/cockroach_versions.go | 8 + pkg/cmd/generate-logictest/templates.go | 3 + pkg/cmd/roachtest/tests/multitenant_tpch.go | 21 +- pkg/col/coldataext/extended_column_factory.go | 8 + pkg/col/colserde/arrowbatchconverter.go | 2 + pkg/kv/kvserver/batcheval/cmd_reverse_scan.go | 9 + pkg/kv/kvserver/batcheval/cmd_scan.go | 9 + pkg/kv/kvserver/batcheval/intent.go | 2 + pkg/kv/kvserver/replica_read.go | 3 + pkg/kv/kvserver/replica_tscache.go | 4 + pkg/roachpb/BUILD.bazel | 2 + pkg/roachpb/api.proto | 66 ++- pkg/roachpb/batch.go | 28 ++ pkg/sql/colexec/colbuilder/BUILD.bazel | 3 + pkg/sql/colexec/colbuilder/execplan.go | 91 +++- pkg/sql/colfetcher/BUILD.bazel | 5 + pkg/sql/colfetcher/cfetcher.go | 171 ++++--- pkg/sql/colfetcher/cfetcher_setup.go | 24 +- pkg/sql/colfetcher/cfetcher_wrapper.go | 200 ++++++++ pkg/sql/colfetcher/colbatch_direct_scan.go | 197 ++++++++ pkg/sql/colfetcher/colbatch_scan.go | 323 ++++++------ pkg/sql/colfetcher/index_join.go | 2 +- .../colfetcher/vectorized_batch_size_test.go | 16 +- .../colflow/vectorized_flow_planning_test.go | 6 +- pkg/sql/exec_util.go | 4 + pkg/sql/explain_bundle.go | 11 + pkg/sql/logictest/logic.go | 10 + .../testdata/logic_test/dist_vectorize | 10 +- .../testdata/logic_test/distsql_enum | 14 +- .../testdata/logic_test/information_schema | 3 +- .../logictest/testdata/logic_test/pg_catalog | 5 +- .../logictest/testdata/logic_test/show_source | 2 +- .../opt/exec/execbuilder/testdata/explain_env | 32 +- .../execbuilder/tests/5node/generated_test.go | 3 + .../tests/fakedist-disk/generated_test.go | 3 + .../tests/fakedist-vec-off/generated_test.go | 3 + .../tests/fakedist/generated_test.go | 3 + .../generated_test.go | 3 + .../tests/local-vec-off/generated_test.go | 3 + .../execbuilder/tests/local/generated_test.go | 3 + pkg/sql/row/BUILD.bazel | 1 + pkg/sql/row/fetcher.go | 36 +- pkg/sql/row/kv_batch_fetcher.go | 41 +- pkg/sql/row/kv_batch_streamer.go | 5 +- pkg/sql/row/kv_fetcher.go | 98 +++- pkg/sql/row/locking.go | 4 +- pkg/sql/sessiondatapb/session_data.proto | 4 + pkg/sql/trace_test.go | 5 + pkg/sql/vars.go | 19 + pkg/storage/BUILD.bazel | 1 + pkg/storage/col_mvcc.go | 472 +++++++++++++++++- pkg/storage/mvcc.go | 65 ++- pkg/storage/pebble_mvcc_scanner.go | 111 ++-- 57 files changed, 1781 insertions(+), 406 deletions(-) create mode 100644 pkg/sql/colfetcher/cfetcher_wrapper.go create mode 100644 pkg/sql/colfetcher/colbatch_direct_scan.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 1d256d3e947e..529b3cafc7d2 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -297,4 +297,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000022.2-30 set the active cluster version in the format '.' +version version 1000022.2-32 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index bb9b50924c4f..0ef19edc2bac 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -235,6 +235,6 @@
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. -
version
version1000022.2-30set the active cluster version in the format '<major>.<minor>' +
version
version1000022.2-32set the active cluster version in the format '<major>.<minor>' diff --git a/pkg/ccl/logictestccl/tests/3node-tenant-multiregion/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant-multiregion/generated_test.go index 7ee218605643..3520c2126eed 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant-multiregion/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant-multiregion/generated_test.go @@ -73,6 +73,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { skip.UnderDeadlock(t, "times out and/or hangs") serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index 6d8f99d14179..f3bb39963f97 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -87,6 +87,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { skip.UnderDeadlock(t, "times out and/or hangs") serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 92bb5cc6d747..b77f71e44adc 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -401,6 +401,10 @@ const ( // chagnefeeds created prior to this version. V23_1_ChangefeedExpressionProductionReady + // V23_1_KVDirectColumnarScans introduces the support of the "direct" + // columnar scans in the KV layer. + V23_1_KVDirectColumnarScans + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -687,6 +691,10 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_1_ChangefeedExpressionProductionReady, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 30}, }, + { + Key: V23_1_KVDirectColumnarScans, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 32}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/cmd/generate-logictest/templates.go b/pkg/cmd/generate-logictest/templates.go index abf9e727696d..dacea0f2caef 100644 --- a/pkg/cmd/generate-logictest/templates.go +++ b/pkg/cmd/generate-logictest/templates.go @@ -74,6 +74,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true,{{ if .ForceProductionValues }} ForceProductionValues: true,{{end}} + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/cmd/roachtest/tests/multitenant_tpch.go b/pkg/cmd/roachtest/tests/multitenant_tpch.go index 45905b97745f..fd3886b4ad83 100644 --- a/pkg/cmd/roachtest/tests/multitenant_tpch.go +++ b/pkg/cmd/roachtest/tests/multitenant_tpch.go @@ -26,7 +26,9 @@ import ( // runMultiTenantTPCH runs TPCH queries on a cluster that is first used as a // single-tenant deployment followed by a run of all queries in a multi-tenant // deployment with a single SQL instance. -func runMultiTenantTPCH(ctx context.Context, t test.Test, c cluster.Cluster) { +func runMultiTenantTPCH( + ctx context.Context, t test.Test, c cluster.Cluster, enableDirectScans bool, +) { c.Put(ctx, t.Cockroach(), "./cockroach", c.All()) c.Put(ctx, t.DeprecatedWorkload(), "./workload", c.Node(1)) c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(install.SecureOption(true)), c.All()) @@ -40,6 +42,11 @@ func runMultiTenantTPCH(ctx context.Context, t test.Test, c cluster.Cluster) { // one at a time (using the given url as a parameter to the 'workload run' // command). The runtimes are accumulated in the perf helper. runTPCH := func(conn *gosql.DB, url string, setupIdx int) { + setting := fmt.Sprintf("SET CLUSTER SETTING sql.distsql.direct_columnar_scans.enabled = %t", enableDirectScans) + t.Status(setting) + if _, err := conn.Exec(setting); err != nil { + t.Fatal(err) + } t.Status("restoring TPCH dataset for Scale Factor 1 in ", setupNames[setupIdx]) if err := loadTPCHDataset( ctx, t, c, conn, 1 /* sf */, c.NewMonitor(ctx), c.All(), false, /* disableMergeQueue */ @@ -93,6 +100,16 @@ func registerMultiTenantTPCH(r registry.Registry) { Name: "multitenant/tpch", Owner: registry.OwnerSQLQueries, Cluster: r.MakeClusterSpec(1 /* nodeCount */), - Run: runMultiTenantTPCH, + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runMultiTenantTPCH(ctx, t, c, false /* enableDirectScans */) + }, + }) + r.Add(registry.TestSpec{ + Name: "multitenant/tpch_direct_scans", + Owner: registry.OwnerSQLQueries, + Cluster: r.MakeClusterSpec(1 /* nodeCount */), + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) { + runMultiTenantTPCH(ctx, t, c, true /* enableDirectScans */) + }, }) } diff --git a/pkg/col/coldataext/extended_column_factory.go b/pkg/col/coldataext/extended_column_factory.go index b3d462f0f4ee..ba009034f654 100644 --- a/pkg/col/coldataext/extended_column_factory.go +++ b/pkg/col/coldataext/extended_column_factory.go @@ -32,6 +32,14 @@ func NewExtendedColumnFactory(evalCtx *eval.Context) coldata.ColumnFactory { return &extendedColumnFactory{evalCtx: evalCtx} } +// NewExtendedColumnFactoryNoEvalCtx returns an extendedColumnFactory that will +// be producing coldata.DatumVecs that aren't fully initialized - the eval +// context is not set on those vectors. This can be acceptable if the caller +// cannot provide the eval.Context but also doesn't intend to compare datums. +func NewExtendedColumnFactoryNoEvalCtx() coldata.ColumnFactory { + return &extendedColumnFactory{} +} + func (cf *extendedColumnFactory) MakeColumn(t *types.T, n int) coldata.Column { if typeconv.TypeFamilyToCanonicalTypeFamily(t.Family()) == typeconv.DatumVecCanonicalTypeFamily { return newDatumVec(t, n, cf.evalCtx) diff --git a/pkg/col/colserde/arrowbatchconverter.go b/pkg/col/colserde/arrowbatchconverter.go index 8e296cc70c36..deab82c2489b 100644 --- a/pkg/col/colserde/arrowbatchconverter.go +++ b/pkg/col/colserde/arrowbatchconverter.go @@ -569,6 +569,8 @@ func getValueBytesAndOffsets( // Release should be called once the converter is no longer needed so that its // memory could be GCed. +// TODO(yuzefovich): consider renaming this to Close in order to not be confused +// with execreleasable.Releasable interface. func (c *ArrowBatchConverter) Release(ctx context.Context) { if c.acc != nil { c.acc.Shrink(ctx, c.accountedFor) diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index 8ae129f7976e..f438f9299e52 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -63,6 +63,15 @@ func ReverseScan( return result.Result{}, err } reply.BatchResponses = scanRes.KVData + case roachpb.COL_BATCH_RESPONSE: + scanRes, err = storage.MVCCScanToCols( + ctx, reader, cArgs.Header.IndexFetchSpec, args.Key, args.EndKey, + h.Timestamp, opts, cArgs.EvalCtx.ClusterSettings(), + ) + if err != nil { + return result.Result{}, err + } + reply.BatchResponses = scanRes.KVData case roachpb.KEY_VALUES: scanRes, err = storage.MVCCScan( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index 19f2f6aa48c4..d4817ec92823 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -64,6 +64,15 @@ func Scan( return result.Result{}, err } reply.BatchResponses = scanRes.KVData + case roachpb.COL_BATCH_RESPONSE: + scanRes, err = storage.MVCCScanToCols( + ctx, reader, cArgs.Header.IndexFetchSpec, args.Key, args.EndKey, + h.Timestamp, opts, cArgs.EvalCtx.ClusterSettings(), + ) + if err != nil { + return result.Result{}, err + } + reply.BatchResponses = scanRes.KVData case roachpb.KEY_VALUES: scanRes, err = storage.MVCCScan( ctx, reader, args.Key, args.EndKey, h.Timestamp, opts) diff --git a/pkg/kv/kvserver/batcheval/intent.go b/pkg/kv/kvserver/batcheval/intent.go index 19f170fb7dd5..523904934c65 100644 --- a/pkg/kv/kvserver/batcheval/intent.go +++ b/pkg/kv/kvserver/batcheval/intent.go @@ -125,6 +125,8 @@ func acquireUnreplicatedLocksOnKeys( res.Local.AcquiredLocks[i] = roachpb.MakeLockAcquisition(txn, copyKey(row.Key), lock.Unreplicated) } return nil + case roachpb.COL_BATCH_RESPONSE: + return errors.AssertionFailedf("unexpectedly acquiring unreplicated locks with COL_BATCH_RESPONSE scan format") default: panic("unexpected scanFormat") } diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index b7a46e4c9301..2ee3afd91a4d 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -531,6 +531,9 @@ func (r *Replica) collectSpansRead( resp := br.Responses[i].GetInner() if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + if ba.IndexFetchSpec != nil { + return nil, nil, errors.AssertionFailedf("unexpectedly IndexFetchSpec is set with SKIP LOCKED wait policy") + } // If the request is using a SkipLocked wait policy, it behaves as if run // at a lower isolation level for any keys that it skips over. If the read // request did not return a key, it does not need to check for conflicts diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 58f22029b9a1..49014a5a5789 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -99,6 +100,9 @@ func (r *Replica) updateTimestampCache( start, end := header.Key, header.EndKey if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && roachpb.CanSkipLocked(req) { + if ba.IndexFetchSpec != nil { + log.Errorf(ctx, "%v", errors.AssertionFailedf("unexpectedly IndexFetchSpec is set with SKIP LOCKED wait policy")) + } // If the request is using a SkipLocked wait policy, it behaves as if run // at a lower isolation level for any keys that it skips over. If the read // request did not return a key, it does not make a claim about whether diff --git a/pkg/roachpb/BUILD.bazel b/pkg/roachpb/BUILD.bazel index 49b89ef2af00..e9e9b53acd46 100644 --- a/pkg/roachpb/BUILD.bazel +++ b/pkg/roachpb/BUILD.bazel @@ -151,6 +151,7 @@ proto_library( "//pkg/kv/kvserver/concurrency/lock:lock_proto", "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", "//pkg/settings:settings_proto", + "//pkg/sql/catalog/fetchpb:fetchpb_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/util:util_proto", "//pkg/util/admission/admissionpb:admissionpb_proto", @@ -173,6 +174,7 @@ go_proto_library( "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/readsummary/rspb", "//pkg/settings", + "//pkg/sql/catalog/fetchpb", "//pkg/storage/enginepb", "//pkg/util", "//pkg/util/admission/admissionpb", diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 59b8c183380b..6030f00b5642 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -20,6 +20,7 @@ import "roachpb/errors.proto"; import "roachpb/metadata.proto"; import "roachpb/span_config.proto"; import "settings/encoding.proto"; +import "sql/catalog/fetchpb/index_fetch.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/mvcc3.proto"; import "util/hlc/timestamp.proto"; @@ -557,6 +558,10 @@ enum ScanFormat { // The batch_response format: a byte slice of alternating keys and values, // each prefixed by their length as a varint. BATCH_RESPONSE = 1; + // The serialized (in the Apache Arrow format) representation of + // coldata.Batch'es that contain only necessary (according to the + // fetchpb.IndexFetchSpec) columns. + COL_BATCH_RESPONSE = 2; } @@ -568,9 +573,9 @@ message ScanRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // The desired format for the response. If set to BATCH_RESPONSE, the server - // will set the batch_responses field in the ScanResponse instead of the rows - // field. + // The desired format for the response. If set to BATCH_RESPONSE or + // COL_BATCH_RESPONSE, the server will set the batch_responses field in the + // ScanResponse instead of the rows field. ScanFormat scan_format = 4; // The desired key-level locking mode used during this scan. When set to None @@ -601,12 +606,19 @@ message ScanResponse { // revisit this decision if this ever becomes a problem. repeated KeyValue intent_rows = 3 [(gogoproto.nullable) = false]; - // If set, each item in this repeated bytes field contains part of the results - // in batch format - the key/value pairs are a buffer of varint-prefixed - // slices, alternating from key to value. Each entry in this field is - // complete - there are no key/value pairs that are split across more than one - // entry. There are num_keys total pairs across all entries, as defined by the - // ResponseHeader. If set, rows will not be set and vice versa. + // If set, then depending on the ScanFormat, each item in this repeated bytes + // field contains part of the results in batch format: + // - for BATCH_RESPONSE - the key/value pairs are a buffer of varint-prefixed + // slices, alternating from key to value. Each entry in this field is complete + // (i.e. there are no key/value pairs that are split across more than one + // entry). There are num_keys total pairs across all entries, as defined by + // the ResponseHeader. + // - for COL_BATCH_RESPONSE - each []byte is a single serialized (in the + // Apache Arrow format) coldata.Batch. Each SQL row in that coldata.Batch is + // complete. num_keys total key-value pairs were used to populate all of the + // coldata.Batch'es in this field. + // + // If set, rows will not be set and vice versa. repeated bytes batch_responses = 4; } @@ -618,9 +630,9 @@ message ReverseScanRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; - // The desired format for the response. If set to BATCH_RESPONSE, the server - // will set the batch_responses field in the ScanResponse instead of the rows - // field. + // The desired format for the response. If set to BATCH_RESPONSE or + // COL_BATCH_RESPONSE, the server will set the batch_responses field in the + // ScanResponse instead of the rows field. ScanFormat scan_format = 4; // The desired key-level locking mode used during this scan. When set to None @@ -651,12 +663,19 @@ message ReverseScanResponse { // revisit this decision if this ever becomes a problem. repeated KeyValue intent_rows = 3 [(gogoproto.nullable) = false]; - // If set, each item in this repeated bytes field contains part of the results - // in batch format - the key/value pairs are a buffer of varint-prefixed - // slices, alternating from key to value. Each entry in this field is - // complete - there are no key/value pairs that are split across more than one - // entry. There are num_keys total pairs across all entries, as defined by the - // ResponseHeader. If set, rows will not be set and vice versa. + // If set, then depending on the ScanFormat, each item in this repeated bytes + // field contains part of the results in batch format: + // - for BATCH_RESPONSE - the key/value pairs are a buffer of varint-prefixed + // slices, alternating from key to value. Each entry in this field is complete + // (i.e. there are no key/value pairs that are split across more than one + // entry). There are num_keys total pairs across all entries, as defined by + // the ResponseHeader. + // - for COL_BATCH_RESPONSE - each []byte is a single serialized (in the + // Apache Arrow format) coldata.Batch. Each SQL row in that coldata.Batch is + // complete. num_keys total key-value pairs were used to populate all of the + // coldata.Batch'es in this field. + // + // If set, rows will not be set and vice versa. repeated bytes batch_responses = 4; } @@ -2523,8 +2542,6 @@ message Header { // The given value specifies the maximum number of keys in a row (i.e. the // number of column families). If any larger rows are found at the end of the // result, an error is returned. - // - // Added in 22.1, callers must check the ScanWholeRows version gate first. int32 whole_rows_of_size = 26; // If true, allow returning an empty result when the first result exceeds a // limit (e.g. TargetBytes). Only supported by Get, Scan, and ReverseScan. @@ -2603,6 +2620,15 @@ message Header { // If a recording mode is set, the response will have collect_spans filled in. util.tracing.tracingpb.TraceInfo trace_info = 25; + // index_fetch_spec, if set, is used for ScanRequests and ReverseScanRequests + // with COL_BATCH_RESPONSE ScanFormat. + // + // The rationale for having this as a single "global" field on the + // BatchRequest is that SQL never issues requests touching different indexes + // in a single BatchRequest, so it would be redundant to copy this field into + // each Scan and ReverseScan. + sql.sqlbase.IndexFetchSpec index_fetch_spec = 29; + reserved 7, 10, 12, 14, 20; } diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index e61fcbc1bc0b..7f9c13ff95d6 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -500,6 +501,9 @@ func (ba *BatchRequest) RefreshSpanIterate(br *BatchResponse, fn func(Span)) err resp = br.Responses[i].GetInner() } if ba.WaitPolicy == lock.WaitPolicy_SkipLocked && CanSkipLocked(req) { + if ba.IndexFetchSpec != nil { + return errors.AssertionFailedf("unexpectedly IndexFetchSpec is set with SKIP LOCKED wait policy") + } // If the request is using a SkipLocked wait policy, it behaves as if run // at a lower isolation level for any keys that it skips over. For this // reason, the request only adds point reads for the individual keys @@ -553,6 +557,8 @@ func ActualSpan(req Request, resp Response) (Span, bool) { // ResponseKeyIterate calls the passed function with the keys returned // in the provided request's response. If no keys are being returned, // the function will not be called. +// NOTE: it is assumed that req (if it is a Scan or a ReverseScan) didn't use +// COL_BATCH_RESPONSE scan format. func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { if resp == nil { return nil @@ -582,6 +588,17 @@ func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { }); err != nil { return err } + if buildutil.CrdbTestBuild { + // COL_BATCH_RESPONSE scan format is not supported. The caller is + // responsible for ensuring this, so this assertion is hidden behind + // the test-build flag. + if req.(*ScanRequest).ScanFormat == COL_BATCH_RESPONSE { + return errors.AssertionFailedf( + "unexpectedly called ResponseKeyIterate on a " + + "ScanRequest with COL_BATCH_RESPONSE scan format", + ) + } + } case *ReverseScanResponse: // If ScanFormat == KEY_VALUES. for _, kv := range v.Rows { @@ -595,6 +612,17 @@ func ResponseKeyIterate(req Request, resp Response, fn func(Key)) error { }); err != nil { return err } + if buildutil.CrdbTestBuild { + // COL_BATCH_RESPONSE scan format is not supported. The caller is + // responsible for ensuring this, so this assertion is hidden behind + // the test-build flag. + if req.(*ReverseScanRequest).ScanFormat == COL_BATCH_RESPONSE { + return errors.AssertionFailedf( + "unexpectedly called ResponseKeyIterate on a " + + "ReverseScanRequest with COL_BATCH_RESPONSE scan format", + ) + } + } default: return errors.Errorf("cannot iterate over response keys of %s request", req.Method()) } diff --git a/pkg/sql/colexec/colbuilder/BUILD.bazel b/pkg/sql/colexec/colbuilder/BUILD.bazel index a3f0ff067a6b..6e73f609a5fe 100644 --- a/pkg/sql/colexec/colbuilder/BUILD.bazel +++ b/pkg/sql/colexec/colbuilder/BUILD.bazel @@ -7,9 +7,11 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/colexec/colbuilder", visibility = ["//visibility:public"], deps = [ + "//pkg/clusterversion", "//pkg/col/coldata", "//pkg/col/coldataext", "//pkg/col/typeconv", + "//pkg/kv/kvserver/concurrency/lock", "//pkg/settings", "//pkg/sql/catalog/descpb", "//pkg/sql/colconv", @@ -31,6 +33,7 @@ go_library( "//pkg/sql/execinfra", "//pkg/sql/execinfra/execreleasable", "//pkg/sql/execinfrapb", + "//pkg/sql/row", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treebin", diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 095ab99e8061..8dd68e17444b 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -15,9 +15,11 @@ import ( "reflect" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/col/coldataext" "github.com/cockroachdb/cockroach/pkg/col/typeconv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/colconv" @@ -39,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfra/execreleasable" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treebin" @@ -843,14 +846,88 @@ func NewColOperator( ctx, flowCtx, "cfetcher" /* opName */, spec.ProcessorID, 2, /* numAccounts */ ) estimatedRowCount := spec.EstimatedRowCount - scanOp, err := colfetcher.NewColBatchScan( - ctx, colmem.NewAllocator(ctx, accounts[0], factory), accounts[1], - flowCtx, core.TableReader, post, estimatedRowCount, args.TypeResolver, - ) - if err != nil { - return r, err + var scanOp colfetcher.ScanOperator + var resultTypes []*types.T + if flowCtx.EvalCtx.SessionData().DirectColumnarScansEnabled { + canUseDirectScan := func() bool { + if !flowCtx.EvalCtx.Settings.Version.IsActive(ctx, clusterversion.V23_1_KVDirectColumnarScans) { + return false + } + // We currently don't use the direct scans if TraceKV is + // enabled (due to not being able to tell the KV server + // about it). One idea would be to include this boolean into + // the fetchpb.IndexFetchSpec. + // TODO(yuzefovich, 23.1): support TraceKV option. + if flowCtx.TraceKV { + return false + } + // The current implementation of non-default locking + // strength as well as of SKIP LOCKED wait policy require + // being able to access to the full keys after the + // corresponding request is evaluated. This is not possible, + // in general case, when using the direct scans since only + // needed columns are included into the response. + // TODO(yuzefovich): support non-default locking strength + // and SKIP LOCKED wait policy somehow (#92950). One idea + // would be to simply include all key columns into the set + // of needed for the fetch and to project them away in the + // ColBatchDirectScan. + if row.GetKeyLockingStrength(core.TableReader.LockingStrength) != lock.None || + core.TableReader.LockingWaitPolicy == descpb.ScanLockingWaitPolicy_SKIP_LOCKED { + return false + } + // At the moment, the ColBatchDirectScan cannot handle Gets + // (it's not clear whether it is worth to handle them via + // the same path as for Scans and ReverseScans (which could + // have too large of an overhead) or by teaching the + // operator to also decode a single KV (similar to what + // regular ColBatchScan does)). + // TODO(yuzefovich, 23.1): explore supporting Gets somehow. + for i := range core.TableReader.Spans { + if len(core.TableReader.Spans[i].EndKey) == 0 { + return false + } + } + fetchSpec := core.TableReader.FetchSpec + // Handling user-defined types requires type hydration which + // we cannot easily do on the KV server side, so for the + // time being we disable the direct scans with such types. + // However, we allow for enums to be processed by treating + // them as bytes values. + // TODO(yuzefovich): consider supporting non-enum UDTs + // (#92954). + for _, c := range fetchSpec.KeyAndSuffixColumns { + if c.Type.UserDefined() && c.Type.Family() != types.EnumFamily { + return false + } + } + for _, c := range fetchSpec.FetchedColumns { + if c.Type.UserDefined() && c.Type.Family() != types.EnumFamily { + return false + } + } + return true + } + if canUseDirectScan() { + scanOp, resultTypes, err = colfetcher.NewColBatchDirectScan( + ctx, colmem.NewAllocator(ctx, accounts[0], factory), accounts[1], + flowCtx, core.TableReader, post, args.TypeResolver, + ) + if err != nil { + return r, err + } + } + } + if scanOp == nil { + scanOp, resultTypes, err = colfetcher.NewColBatchScan( + ctx, colmem.NewAllocator(ctx, accounts[0], factory), accounts[1], + flowCtx, core.TableReader, post, estimatedRowCount, args.TypeResolver, + ) + if err != nil { + return r, err + } } - result.finishScanPlanning(scanOp, scanOp.ResultTypes) + result.finishScanPlanning(scanOp, resultTypes) case core.JoinReader != nil: if err := checkNumIn(inputs, 1); err != nil { diff --git a/pkg/sql/colfetcher/BUILD.bazel b/pkg/sql/colfetcher/BUILD.bazel index dc8a3ecef143..b099f5cb95c5 100644 --- a/pkg/sql/colfetcher/BUILD.bazel +++ b/pkg/sql/colfetcher/BUILD.bazel @@ -7,6 +7,8 @@ go_library( srcs = [ "cfetcher.go", "cfetcher_setup.go", + "cfetcher_wrapper.go", + "colbatch_direct_scan.go", "colbatch_scan.go", "index_join.go", ":gen-fetcherstate-stringer", # keep @@ -15,6 +17,8 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/col/coldata", + "//pkg/col/coldataext", + "//pkg/col/colserde", "//pkg/col/typeconv", "//pkg/keys", "//pkg/kv", @@ -59,6 +63,7 @@ go_library( "//pkg/util/syncutil", "//pkg/util/timeutil", "//pkg/util/tracing", + "@com_github_apache_arrow_go_arrow//array", "@com_github_cockroachdb_apd_v3//:apd", "@com_github_cockroachdb_errors//:errors", "@com_github_lib_pq//oid", diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 37e7d86e2802..445c1e9c0551 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -225,6 +225,9 @@ type cFetcher struct { nextKVer storage.NextKVer // fetcher, if set, is the same object as nextKVer. fetcher *row.KVFetcher + // stableKVs indicates whether the KVs returned by nextKVer are stable (i.e. + // are not invalidated) across NextKV() calls. + stableKVs bool // bytesRead and batchRequestsIssued store the total number of bytes read // and of BatchRequests issued, respectively, by this cFetcher throughout // its lifetime in case when the underlying row.KVFetcher has already been @@ -265,6 +268,8 @@ type cFetcher struct { // keys are compared against this prefix to determine whether they're part // of a new row or not. lastRowPrefix roachpb.Key + // firstKeyOfRow, if set, is the first key in the current row. + firstKeyOfRow roachpb.Key // prettyValueBuf is a temp buffer used to create strings for tracing. prettyValueBuf *bytes.Buffer @@ -283,9 +288,11 @@ type cFetcher struct { tableoidCol coldata.DatumVec } - // scratch is a scratch space used when decoding bytes-like and decimal - // keys. - scratch []byte + scratch struct { + decoding []byte + nextKVKey []byte + nextKVRawBytes []byte + } accountingHelper colmem.SetAccountingHelper } @@ -420,12 +427,12 @@ func (cf *cFetcher) Init( indexColOrdinals[i] = -1 } } - if needToDecodeDecimalKey && cap(cf.scratch) < 64 { + if needToDecodeDecimalKey && cap(cf.scratch.decoding) < 64 { // If we need to decode the decimal key encoding, it might use a scratch // byte slice internally, so we'll allocate such a space to be reused // for every decimal. // TODO(yuzefovich): 64 was chosen arbitrarily, tune it. - cf.scratch = make([]byte, 64) + cf.scratch.decoding = make([]byte, 64) } // Unique secondary indexes contain the extra column IDs as part of // the value component. We process these separately, so we need to know @@ -475,35 +482,18 @@ func (cf *cFetcher) Init( if kvFetcher, ok := nextKVer.(*row.KVFetcher); ok { cf.fetcher = kvFetcher } + cf.stableKVs = nextKVer.Init(cf.getFirstKeyOfRow) cf.accountingHelper.Init(allocator, cf.memoryLimit, cf.table.typs) if cf.cFetcherArgs.collectStats { cf.cpuStopWatch = timeutil.NewCPUStopWatch() } + cf.machine.state[0] = stateResetBatch + cf.machine.state[1] = stateInitFetch return nil } -// StartScan initializes and starts the key-value scan. Can only be used -// multiple times if cFetcherArgs.singleUse was set to false in Init(). -// -// The fetcher takes ownership of the spans slice - it can modify the slice and -// will perform the memory accounting accordingly. The caller can only reuse the -// spans slice after the fetcher emits a zero-length batch, and if the caller -// does, it becomes responsible for the memory accounting. -func (cf *cFetcher) StartScan( - ctx context.Context, - spans roachpb.Spans, - limitBatches bool, - batchBytesLimit rowinfra.BytesLimit, - limitHint rowinfra.RowLimit, -) error { - if len(spans) == 0 { - return errors.AssertionFailedf("no spans") - } - if !limitBatches && batchBytesLimit != rowinfra.NoBytesLimit { - return errors.AssertionFailedf("batchBytesLimit set without limitBatches") - } - +func cFetcherFirstBatchLimit(limitHint rowinfra.RowLimit, maxKeysPerRow uint32) rowinfra.KeyLimit { // If we have a limit hint, we limit the first batch size. Subsequent // batches get larger to avoid making things too slow (e.g. in case we have // a very restrictive filter and actually have to retrieve a lot of rows). @@ -526,9 +516,33 @@ func (cf *cFetcher) StartScan( // - KVs for some column families are omitted for some rows - then we // will actually fetch more KVs than necessary, but we'll decode // limitHint number of rows. - firstBatchLimit = rowinfra.KeyLimit(int(limitHint) * int(cf.table.spec.MaxKeysPerRow)) + firstBatchLimit = rowinfra.KeyLimit(int(limitHint) * int(maxKeysPerRow)) + } + return firstBatchLimit +} + +// StartScan initializes and starts the key-value scan. Can only be used +// multiple times if cFetcherArgs.singleUse was set to false in Init(). +// +// The fetcher takes ownership of the spans slice - it can modify the slice and +// will perform the memory accounting accordingly. The caller can only reuse the +// spans slice after the fetcher emits a zero-length batch, and if the caller +// does, it becomes responsible for the memory accounting. +func (cf *cFetcher) StartScan( + ctx context.Context, + spans roachpb.Spans, + limitBatches bool, + batchBytesLimit rowinfra.BytesLimit, + limitHint rowinfra.RowLimit, +) error { + if len(spans) == 0 { + return errors.AssertionFailedf("no spans") + } + if !limitBatches && batchBytesLimit != rowinfra.NoBytesLimit { + return errors.AssertionFailedf("batchBytesLimit set without limitBatches") } + firstBatchLimit := cFetcherFirstBatchLimit(limitHint, cf.table.spec.MaxKeysPerRow) cf.machine.lastRowPrefix = nil cf.machine.limitHint = int(limitHint) cf.machine.state[0] = stateResetBatch @@ -609,27 +623,39 @@ func (cf *cFetcher) setEstimatedRowCount(estimatedRowCount uint64) { cf.estimatedRowCount = estimatedRowCount } -// setNextKV sets the next KV to process to the input KV. needsCopy, if true, -// causes the input kv to be deep copied. needsCopy should be set to true if -// the input KV is pointing to the last KV of a batch, so that the batch can -// be garbage collected before fetching the next one. +func (cf *cFetcher) getFirstKeyOfRow() roachpb.Key { + return cf.machine.firstKeyOfRow +} + +// setNextKV sets the next KV to process to the input KV. The KV will be +// deep-copied if necessary, however, the copy is only valid until the next +// setNextKV call. // gcassert:inline -func (cf *cFetcher) setNextKV(kv roachpb.KeyValue, needsCopy bool) { - if !needsCopy { +func (cf *cFetcher) setNextKV(kv roachpb.KeyValue) { + // If the kv is not stable and the table has multiple column families, then + // we must perform a deep copy. This is due to the fact that we keep a + // shallow reference to the first KV of each row (in + // cf.machine.lastRowPrefix and cf.machine.firstKeyOfRow). + // + // However, even if the kv is not stable, but there is only one column + // family, then we will have finalized the row (meaning we'll have deep + // copied necessary part of the kv into the batch) by the time NextKV is + // called again, so we avoid the copy in those cases. + if cf.stableKVs || cf.table.spec.MaxKeysPerRow == 1 { cf.machine.nextKV = kv return } - - // If we've made it to the very last key in the batch, copy out the key - // so that the GC can reclaim the large backing slice before we call - // NextKV() again. - kvCopy := roachpb.KeyValue{} - kvCopy.Key = make(roachpb.Key, len(kv.Key)) - copy(kvCopy.Key, kv.Key) - kvCopy.Value.RawBytes = make([]byte, len(kv.Value.RawBytes)) - copy(kvCopy.Value.RawBytes, kv.Value.RawBytes) - kvCopy.Value.Timestamp = kv.Value.Timestamp - cf.machine.nextKV = kvCopy + // We can reuse the scratch space since we only need to keep at most one KV + // at a time. + cf.scratch.nextKVKey = append(cf.scratch.nextKVKey[:0], kv.Key...) + cf.scratch.nextKVRawBytes = append(cf.scratch.nextKVRawBytes[:0], kv.Value.RawBytes...) + cf.machine.nextKV = roachpb.KeyValue{ + Key: cf.scratch.nextKVKey, + Value: roachpb.Value{ + RawBytes: cf.scratch.nextKVRawBytes, + Timestamp: kv.Value.Timestamp, + }, + } } // NextBatch processes keys until we complete one batch of rows (subject to the @@ -648,11 +674,15 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { case stateInvalid: return nil, errors.New("invalid fetcher state") case stateInitFetch: + cf.machine.firstKeyOfRow = nil cf.cpuStopWatch.Start() - moreKVs, kv, needsCopy, err := cf.nextKVer.NextKV(ctx, cf.mvccDecodeStrategy) + // Here we ignore partialRow return parameter because it can only be + // true when moreKVs is false, in which case we have already + // finalized the last row and will emit the batch as is. + moreKVs, _, kv, err := cf.nextKVer.NextKV(ctx, cf.mvccDecodeStrategy) cf.cpuStopWatch.Stop() if err != nil { - return nil, cf.convertFetchError(ctx, err) + return nil, convertFetchError(&cf.table.spec, err) } if !moreKVs { cf.machine.state[0] = stateEmitLastBatch @@ -680,7 +710,7 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { } */ - cf.setNextKV(kv, needsCopy) + cf.setNextKV(kv) cf.machine.state[0] = stateDecodeFirstKVOfRow case stateResetBatch: @@ -689,6 +719,7 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { case stateDecodeFirstKVOfRow: // Reset MVCC metadata for the table, since this is the first KV of a row. cf.table.rowLastModified = hlc.Timestamp{} + cf.machine.firstKeyOfRow = cf.machine.nextKV.Key // foundNull is set when decoding a new index key for a row finds a NULL value // in the index key. This is used when decoding unique secondary indexes in order @@ -707,7 +738,7 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { // to determine whether a KV belongs to the same row as the // previous KV or a different row. checkAllColsForNull := cf.table.spec.IsSecondaryIndex && cf.table.spec.IsUniqueIndex && cf.table.spec.MaxKeysPerRow != 1 - key, foundNull, cf.scratch, err = colencoding.DecodeKeyValsToCols( + key, foundNull, cf.scratch.decoding, err = colencoding.DecodeKeyValsToCols( &cf.table.da, &cf.machine.colvecs, cf.machine.rowIdx, @@ -716,7 +747,7 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { cf.table.spec.KeyFullColumns(), nil, /* unseen */ cf.machine.nextKV.Key[cf.table.spec.KeyPrefixLength:], - cf.scratch, + cf.scratch.decoding, ) if err != nil { return nil, err @@ -801,33 +832,46 @@ func (cf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { case stateFetchNextKVWithUnfinishedRow: cf.cpuStopWatch.Start() - moreKVs, kv, needsCopy, err := cf.nextKVer.NextKV(ctx, cf.mvccDecodeStrategy) + moreKVs, partialRow, kv, err := cf.nextKVer.NextKV(ctx, cf.mvccDecodeStrategy) cf.cpuStopWatch.Stop() if err != nil { - return nil, cf.convertFetchError(ctx, err) + return nil, convertFetchError(&cf.table.spec, err) } if !moreKVs { - // No more data. Finalize the row and exit. + // No more data. + if partialRow { + // The stream of KVs stopped in the middle of the last row, + // so we need to remove that last row from the batch. We + // achieve this by simply not incrementing rowIdx and not + // finalizing this last partial row; instead, we proceed + // straight to emitting the last batch. + cf.machine.state[0] = stateEmitLastBatch + continue + } + // Finalize the row and exit. cf.machine.state[0] = stateFinalizeRow cf.machine.state[1] = stateEmitLastBatch continue } - // TODO(jordan): if nextKV returns newSpan = true, set the new span - // prefix and indicate that it needs decoding. - cf.setNextKV(kv, needsCopy) if debugState { - log.Infof(ctx, "decoding next key %s", cf.machine.nextKV.Key) + log.Infof(ctx, "decoding next key %s", kv.Key) } // TODO(yuzefovich): optimize this prefix check by skipping logical // longest common span prefix. if !bytes.HasPrefix(kv.Key[cf.table.spec.KeyPrefixLength:], cf.machine.lastRowPrefix[cf.table.spec.KeyPrefixLength:]) { // The kv we just found is from a different row. + cf.setNextKV(kv) cf.machine.state[0] = stateFinalizeRow cf.machine.state[1] = stateDecodeFirstKVOfRow continue } + // No need to copy this kv even if it is unstable since we only use + // it before issuing the following NextKV() call (which could + // invalidate it). + cf.machine.nextKV = kv + familyID, err := cf.getCurrentColumnFamilyID() if err != nil { return nil, err @@ -1047,7 +1091,7 @@ func (cf *cFetcher) processValue(ctx context.Context, familyID descpb.FamilyID) if table.spec.IsSecondaryIndex && table.spec.IsUniqueIndex { // This is a unique secondary index; decode the extra // column values from the value. - valueBytes, _, cf.scratch, err = colencoding.DecodeKeyValsToCols( + valueBytes, _, cf.scratch.decoding, err = colencoding.DecodeKeyValsToCols( &table.da, &cf.machine.colvecs, cf.machine.rowIdx, @@ -1056,7 +1100,7 @@ func (cf *cFetcher) processValue(ctx context.Context, familyID descpb.FamilyID) table.spec.KeySuffixColumns(), &cf.machine.remainingValueColsByIdx, valueBytes, - cf.scratch, + cf.scratch.decoding, ) if err != nil { return scrub.WrapError(scrub.SecondaryIndexKeyExtraValueDecodingError, err) @@ -1293,8 +1337,8 @@ func (cf *cFetcher) getCurrentColumnFamilyID() (descpb.FamilyID, error) { // storage error that will propagate through the exec subsystem unchanged. The // error may also undergo a mapping to make it more user friendly for SQL // consumers. -func (cf *cFetcher) convertFetchError(ctx context.Context, err error) error { - err = row.ConvertFetchError(&cf.table.spec, err) +func convertFetchError(indexFetchSpec *fetchpb.IndexFetchSpec, err error) error { + err = row.ConvertFetchError(indexFetchSpec, err) err = colexecerror.NewStorageError(err) return err } @@ -1336,9 +1380,10 @@ func (cf *cFetcher) Release() { } colvecs := cf.machine.colvecs colvecs.Reset() - *cf = cFetcher{ - scratch: cf.scratch[:0], - } + *cf = cFetcher{scratch: cf.scratch} + cf.scratch.decoding = cf.scratch.decoding[:0] + cf.scratch.nextKVKey = cf.scratch.nextKVKey[:0] + cf.scratch.nextKVRawBytes = cf.scratch.nextKVRawBytes[:0] cf.machine.colvecs = colvecs cFetcherPool.Put(cf) } diff --git a/pkg/sql/colfetcher/cfetcher_setup.go b/pkg/sql/colfetcher/cfetcher_setup.go index 48f707f17967..88c7dd0424e8 100644 --- a/pkg/sql/colfetcher/cfetcher_setup.go +++ b/pkg/sql/colfetcher/cfetcher_setup.go @@ -60,8 +60,14 @@ func (a *cFetcherTableArgs) populateTypes(cols []fetchpb.IndexFetchSpec_Column) } // populateTableArgs fills in cFetcherTableArgs. +// - allowUnhydratedEnums, if set, indicates that the type hydration of enums +// should be skipped. This should only be used when the enums will be +// serialized and won't be accessed directly. func populateTableArgs( - ctx context.Context, fetchSpec *fetchpb.IndexFetchSpec, typeResolver *descs.DistSQLTypeResolver, + ctx context.Context, + fetchSpec *fetchpb.IndexFetchSpec, + typeResolver *descs.DistSQLTypeResolver, + allowUnhydratedEnums bool, ) (_ *cFetcherTableArgs, _ error) { args := cFetcherTableArgsPool.Get().(*cFetcherTableArgs) @@ -73,13 +79,21 @@ func populateTableArgs( // they are hydrated. In row execution engine it is done during the processor // initialization, but neither ColBatchScan nor cFetcher are processors, so we // need to do the hydration ourselves. - for i := range args.spec.FetchedColumns { - if err := typedesc.EnsureTypeIsHydrated(ctx, args.spec.FetchedColumns[i].Type, typeResolver); err != nil { + for _, c := range args.spec.FetchedColumns { + t := c.Type + if allowUnhydratedEnums && t.Family() == types.EnumFamily { + continue + } + if err := typedesc.EnsureTypeIsHydrated(ctx, t, typeResolver); err != nil { return nil, err } } - for i := range args.spec.KeyAndSuffixColumns { - if err := typedesc.EnsureTypeIsHydrated(ctx, args.spec.KeyAndSuffixColumns[i].Type, typeResolver); err != nil { + for _, c := range args.spec.KeyAndSuffixColumns { + t := c.Type + if allowUnhydratedEnums && t.Family() == types.EnumFamily { + continue + } + if err := typedesc.EnsureTypeIsHydrated(ctx, t, typeResolver); err != nil { return nil, err } } diff --git a/pkg/sql/colfetcher/cfetcher_wrapper.go b/pkg/sql/colfetcher/cfetcher_wrapper.go new file mode 100644 index 000000000000..3176b836b7a0 --- /dev/null +++ b/pkg/sql/colfetcher/cfetcher_wrapper.go @@ -0,0 +1,200 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colfetcher + +import ( + "bytes" + "context" + + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/coldataext" + "github.com/cockroachdb/cockroach/pkg/col/colserde" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/mon" +) + +// DirectScansEnabled is a cluster setting that controls whether the KV +// projection pushdown infrastructure can be used. +var DirectScansEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.distsql.direct_columnar_scans.enabled", + "set to true to enable the 'direct' columnar scans in the KV layer", + directScansEnabledDefault, +) + +var directScansEnabledDefault = util.ConstantWithMetamorphicTestBool( + "direct-scans-enabled", + // TODO(yuzefovich, 23.1): update the default to 'true' for multi-tenant + // setups. + false, +) + +// cFetcherWrapper implements the storage.CFetcherWrapper interface. See a large +// comment in storage/col_mvcc.go for more details. +type cFetcherWrapper struct { + fetcher *cFetcher + + // adapter is a utility struct needed to wrap a call to fetcher.NextBatch + // with a panic-catcher. + adapter struct { + ctx context.Context + batch coldata.Batch + err error + } + + // startKey is only used as an additional detail for some error messages. + startKey roachpb.Key + + converterAcc *mon.BoundAccount + // TODO(yuzefovich): consider extracting a serializer component that would + // be also reused by the colrpc.Outbox. + converter *colserde.ArrowBatchConverter + serializer *colserde.RecordBatchSerializer + buf bytes.Buffer +} + +var _ storage.CFetcherWrapper = &cFetcherWrapper{} + +func init() { + storage.GetCFetcherWrapper = newCFetcherWrapper +} + +func (c *cFetcherWrapper) nextBatchAdapter() { + c.adapter.batch, c.adapter.err = c.fetcher.NextBatch(c.adapter.ctx) +} + +// NextBatch implements the storage.CFetcherWrapper interface. +func (c *cFetcherWrapper) NextBatch(ctx context.Context) ([]byte, error) { + // cFetcher propagates some errors as "internal" panics, so we have to wrap + // a call to cFetcher.NextBatch with a panic-catcher. + c.adapter.ctx = ctx + if err := colexecerror.CatchVectorizedRuntimeError(c.nextBatchAdapter); err != nil { + // Most likely this error indicates that a memory limit was reached by + // the wrapped cFetcher, so we want to augment it with an additional + // detail about the start key. + return nil, storage.IncludeStartKeyIntoErr(c.startKey, err) + } + if c.adapter.err != nil { + // If an error is propagated in a "regular" fashion, as a return + // parameter, then we don't include the start key - the pebble MVCC + // scanner has already done so if needed. + return nil, c.adapter.err + } + if c.adapter.batch.Length() == 0 { + return nil, nil + } + data, err := c.converter.BatchToArrow(ctx, c.adapter.batch) + if err != nil { + return nil, err + } + oldBufCap := c.buf.Cap() + c.buf.Reset() + _, _, err = c.serializer.Serialize(&c.buf, data, c.adapter.batch.Length()) + if err != nil { + return nil, err + } + if newBufCap := c.buf.Cap(); newBufCap > oldBufCap { + // Account for the capacity of the buffer since we're reusing it across + // NextBatch calls. Note that it is ok to reuse the memory account of + // the ArrowBatchConverter for this since the converter only grows / + // shrinks the account according to its own usage and never relies on + // the total used value. + if err = c.converterAcc.Grow(ctx, int64(newBufCap-oldBufCap)); err != nil { + return nil, err + } + } + return c.buf.Bytes(), nil +} + +// Close implements the storage.CFetcherWrapper interface. +func (c *cFetcherWrapper) Close(ctx context.Context) { + if c.fetcher != nil { + c.fetcher.Close(ctx) + c.fetcher.Release() + c.fetcher = nil + } + if c.converter != nil { + c.converter.Release(ctx) + c.converter = nil + } + c.buf = bytes.Buffer{} +} + +func newCFetcherWrapper( + ctx context.Context, + fetcherAccount *mon.BoundAccount, + converterAccount *mon.BoundAccount, + fetchSpec *fetchpb.IndexFetchSpec, + nextKVer storage.NextKVer, + startKey roachpb.Key, +) (_ storage.CFetcherWrapper, retErr error) { + // At the moment, we always serialize the columnar batches, so it is safe to + // handle enum types without proper hydration - we just treat them as bytes + // values, and it is the responsibility of the ColBatchDirectScan to hydrate + // the type correctly when deserializing the batches. + const allowUnhydratedEnums = true + tableArgs, err := populateTableArgs(ctx, fetchSpec, nil /* typeResolver */, allowUnhydratedEnums) + if err != nil { + return nil, err + } + + fetcher := cFetcherPool.Get().(*cFetcher) + defer func() { + if retErr != nil { + fetcher.Release() + } + }() + // This memory limit determines the maximum footprint of a single batch + // produced by the cFetcher. The main limiting behavior is actually driven + // by the pebbleMVCCScanner (which respects TargetSize and + // MaxSpanRequestKeys limits of the BatchRequest), so we just have a + // reasonable default here. + const memoryLimit = execinfra.DefaultMemoryLimit + // TODO(yuzefovich, 23.1): think through estimatedRowCount (#94850) and + // traceKV arguments. + fetcher.cFetcherArgs = cFetcherArgs{ + memoryLimit, + 0, /* estimatedRowCount */ + false, /* traceKV */ + true, /* singleUse */ + false, /* collectStats */ + } + + // We don't need to provide the eval context here since we will only decode + // bytes into datums and then serialize them, without ever comparing datums + // (at least until we implement the filter pushdown). + allocator := colmem.NewAllocator(ctx, fetcherAccount, coldataext.NewExtendedColumnFactoryNoEvalCtx()) + if err = fetcher.Init(allocator, nextKVer, tableArgs); err != nil { + return nil, err + } + // TODO(yuzefovich, 23.1): consider pooling the allocations of some objects. + wrapper := cFetcherWrapper{ + fetcher: fetcher, + startKey: startKey, + converterAcc: converterAccount, + } + wrapper.converter, err = colserde.NewArrowBatchConverter(tableArgs.typs, colserde.BatchToArrowOnly, converterAccount) + if err != nil { + return nil, err + } + wrapper.serializer, err = colserde.NewRecordBatchSerializer(tableArgs.typs) + if err != nil { + return nil, err + } + return &wrapper, nil +} diff --git a/pkg/sql/colfetcher/colbatch_direct_scan.go b/pkg/sql/colfetcher/colbatch_direct_scan.go new file mode 100644 index 000000000000..934c6820ab55 --- /dev/null +++ b/pkg/sql/colfetcher/colbatch_direct_scan.go @@ -0,0 +1,197 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package colfetcher + +import ( + "context" + "time" + + "github.com/apache/arrow/go/arrow/array" + "github.com/cockroachdb/cockroach/pkg/col/coldata" + "github.com/cockroachdb/cockroach/pkg/col/colserde" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" + "github.com/cockroachdb/cockroach/pkg/sql/colexecerror" + "github.com/cockroachdb/cockroach/pkg/sql/colmem" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/row" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/errors" +) + +// ColBatchDirectScan is a colexecop.Operator that performs a scan of the given +// key spans using the COL_BATCH_RESPONSE scan format. +type ColBatchDirectScan struct { + *colBatchScanBase + fetcher row.KVBatchFetcher + + allocator *colmem.Allocator + spec *fetchpb.IndexFetchSpec + resultTypes []*types.T + + data []array.Data + batch coldata.Batch + converter *colserde.ArrowBatchConverter + deser *colserde.RecordBatchSerializer +} + +var _ ScanOperator = &ColBatchDirectScan{} + +// Init implements the colexecop.Operator interface. +func (s *ColBatchDirectScan) Init(ctx context.Context) { + if !s.InitHelper.Init(ctx) { + return + } + // If tracing is enabled, we need to start a child span so that the only + // contention events present in the recording would be because of this + // fetcher. Note that ProcessorSpan method itself will check whether tracing + // is enabled. + s.Ctx, s.tracingSpan = execinfra.ProcessorSpan(s.Ctx, "colbatchdirectscan") + var err error + s.deser, err = colserde.NewRecordBatchSerializer(s.resultTypes) + if err != nil { + colexecerror.InternalError(err) + } + s.converter, err = colserde.NewArrowBatchConverter(s.resultTypes, colserde.ArrowToBatchOnly, nil /* acc */) + if err != nil { + colexecerror.InternalError(err) + } + firstBatchLimit := cFetcherFirstBatchLimit(s.limitHint, s.spec.MaxKeysPerRow) + err = s.fetcher.SetupNextFetch( + ctx, s.Spans, nil /* spanIDs */, s.batchBytesLimit, firstBatchLimit, + ) + if err != nil { + colexecerror.InternalError(err) + } +} + +// Next implements the colexecop.Operator interface. +func (s *ColBatchDirectScan) Next() (ret coldata.Batch) { + var res row.KVBatchFetcherResponse + var err error + for { + res, err = s.fetcher.NextBatch(s.Ctx) + if err != nil { + colexecerror.InternalError(convertFetchError(s.spec, err)) + } + if !res.MoreKVs { + return coldata.ZeroBatch + } + if res.KVs != nil { + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly encountered KVs in a direct scan")) + } + if res.BatchResponse != nil { + break + } + // If BatchResponse is nil, then it was an empty response for a + // ScanRequest, and we need to proceed further. + } + s.data = s.data[:0] + batchLength, err := s.deser.Deserialize(&s.data, res.BatchResponse) + if err != nil { + colexecerror.InternalError(err) + } + // We rely on the cFetcherWrapper to produce reasonably sized batches. + s.batch, _ = s.allocator.ResetMaybeReallocateNoMemLimit(s.resultTypes, s.batch, batchLength) + s.allocator.PerformOperation(s.batch.ColVecs(), func() { + if err = s.converter.ArrowToBatch(s.data, batchLength, s.batch); err != nil { + colexecerror.InternalError(err) + } + }) + s.mu.Lock() + s.mu.rowsRead += int64(batchLength) + s.mu.Unlock() + return s.batch +} + +// DrainMeta is part of the colexecop.MetadataSource interface. +func (s *ColBatchDirectScan) DrainMeta() []execinfrapb.ProducerMetadata { + trailingMeta := s.colBatchScanBase.drainMeta() + meta := execinfrapb.GetProducerMeta() + meta.Metrics = execinfrapb.GetMetricsMeta() + meta.Metrics.BytesRead = s.GetBytesRead() + meta.Metrics.RowsRead = s.GetRowsRead() + trailingMeta = append(trailingMeta, *meta) + return trailingMeta +} + +// GetBytesRead is part of the colexecop.KVReader interface. +func (s *ColBatchDirectScan) GetBytesRead() int64 { + return s.fetcher.GetBytesRead() +} + +// GetBatchRequestsIssued is part of the colexecop.KVReader interface. +func (s *ColBatchDirectScan) GetBatchRequestsIssued() int64 { + return s.fetcher.GetBatchRequestsIssued() +} + +// GetKVCPUTime is part of the colexecop.KVReader interface. +func (s *ColBatchDirectScan) GetKVCPUTime() time.Duration { + // TODO(yuzefovich, 23.1): implement this. + return 0 +} + +// Release implements the execreleasable.Releasable interface. +func (s *ColBatchDirectScan) Release() { + s.colBatchScanBase.Release() + *s = ColBatchDirectScan{} +} + +// Close implements the colexecop.Closer interface. +func (s *ColBatchDirectScan) Close(context.Context) error { + // Note that we're using the context of the ColBatchDirectScan rather than + // the argument of Close() because the ColBatchDirectScan derives its own + // tracing span. + ctx := s.EnsureCtx() + s.fetcher.Close(ctx) + s.converter.Release(ctx) + return s.colBatchScanBase.close() +} + +// NewColBatchDirectScan creates a new ColBatchDirectScan operator. +func NewColBatchDirectScan( + ctx context.Context, + allocator *colmem.Allocator, + kvFetcherMemAcc *mon.BoundAccount, + flowCtx *execinfra.FlowCtx, + spec *execinfrapb.TableReaderSpec, + post *execinfrapb.PostProcessSpec, + typeResolver *descs.DistSQLTypeResolver, +) (*ColBatchDirectScan, []*types.T, error) { + base, bsHeader, tableArgs, err := newColBatchScanBase( + ctx, kvFetcherMemAcc, flowCtx, spec, post, typeResolver, + ) + if err != nil { + return nil, nil, err + } + fetcher := row.NewDirectKVBatchFetcher( + flowCtx.Txn, + bsHeader, + &spec.FetchSpec, + spec.Reverse, + spec.LockingStrength, + spec.LockingWaitPolicy, + flowCtx.EvalCtx.SessionData().LockTimeout, + kvFetcherMemAcc, + flowCtx.EvalCtx.TestingKnobs.ForceProductionValues, + ) + + return &ColBatchDirectScan{ + colBatchScanBase: base, + allocator: allocator, + fetcher: fetcher, + spec: &spec.FetchSpec, + resultTypes: tableArgs.typs, + data: make([]array.Data, len(tableArgs.typs)), + }, tableArgs.typs, nil +} diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index a34959ccfbb5..fa9574a6315c 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -34,25 +34,14 @@ import ( "github.com/cockroachdb/errors" ) -// TODO(yuzefovich): reading the data through a pair of ColBatchScan and -// materializer turns out to be more efficient than through a table reader (at -// the moment, the exception is the case of reading very small number of rows -// because we still pre-allocate batches of 1024 size). Once we can control the -// initial size of pre-allocated batches (probably via a batch allocator), we -// should get rid off table readers entirely. We will have to be careful about -// propagating the metadata though. - -// ColBatchScan is the exec.Operator implementation of TableReader. It reads a -// table from kv, presenting it as coldata.Batches via the exec.Operator -// interface. -type ColBatchScan struct { +// colBatchScanBase is the common base for ColBatchScan and ColBatchDirectScan +// operators. +type colBatchScanBase struct { colexecop.ZeroInputNode colexecop.InitHelper execinfra.SpansWithCopy flowCtx *execinfra.FlowCtx - bsHeader *roachpb.BoundedStalenessHeader - cf *cFetcher limitHint rowinfra.RowLimit batchBytesLimit rowinfra.BytesLimit parallelize bool @@ -65,61 +54,9 @@ type ColBatchScan struct { // returned so far. rowsRead int64 } - // ResultTypes is the slice of resulting column types from this operator. - // It should be used rather than the slice of column types from the scanned - // table because the scan might synthesize additional implicit system columns. - ResultTypes []*types.T } -// ScanOperator combines common interfaces between operators that perform KV -// scans, such as ColBatchScan and ColIndexJoin. -type ScanOperator interface { - colexecop.KVReader - execreleasable.Releasable - colexecop.ClosableOperator -} - -var _ ScanOperator = &ColBatchScan{} - -// Init initializes a ColBatchScan. -func (s *ColBatchScan) Init(ctx context.Context) { - if !s.InitHelper.Init(ctx) { - return - } - // If tracing is enabled, we need to start a child span so that the only - // contention events present in the recording would be because of this - // cFetcher. Note that ProcessorSpan method itself will check whether - // tracing is enabled. - s.Ctx, s.tracingSpan = execinfra.ProcessorSpan(s.Ctx, "colbatchscan") - limitBatches := !s.parallelize - if err := s.cf.StartScan( - s.Ctx, - s.Spans, - limitBatches, - s.batchBytesLimit, - s.limitHint, - ); err != nil { - colexecerror.InternalError(err) - } -} - -// Next is part of the Operator interface. -func (s *ColBatchScan) Next() coldata.Batch { - bat, err := s.cf.NextBatch(s.Ctx) - if err != nil { - colexecerror.InternalError(err) - } - if bat.Selection() != nil { - colexecerror.InternalError(errors.AssertionFailedf("unexpectedly a selection vector is set on the batch coming from CFetcher")) - } - s.mu.Lock() - s.mu.rowsRead += int64(bat.Length()) - s.mu.Unlock() - return bat -} - -// DrainMeta is part of the colexecop.MetadataSource interface. -func (s *ColBatchScan) DrainMeta() []execinfrapb.ProducerMetadata { +func (s *colBatchScanBase) drainMeta() []execinfrapb.ProducerMetadata { var trailingMeta []execinfrapb.ProducerMetadata if !s.flowCtx.Local { nodeID, ok := s.flowCtx.NodeID.OptionalNodeID() @@ -133,75 +70,65 @@ func (s *ColBatchScan) DrainMeta() []execinfrapb.ProducerMetadata { if tfs := execinfra.GetLeafTxnFinalState(s.Ctx, s.flowCtx.Txn); tfs != nil { trailingMeta = append(trailingMeta, execinfrapb.ProducerMetadata{LeafTxnFinalState: tfs}) } - meta := execinfrapb.GetProducerMeta() - meta.Metrics = execinfrapb.GetMetricsMeta() - meta.Metrics.BytesRead = s.GetBytesRead() - meta.Metrics.RowsRead = s.GetRowsRead() - trailingMeta = append(trailingMeta, *meta) if trace := tracing.SpanFromContext(s.Ctx).GetConfiguredRecording(); trace != nil { trailingMeta = append(trailingMeta, execinfrapb.ProducerMetadata{TraceData: trace}) } return trailingMeta } -// GetBytesRead is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetBytesRead() int64 { - s.mu.Lock() - defer s.mu.Unlock() - return s.cf.getBytesRead() -} - // GetRowsRead is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetRowsRead() int64 { +func (s *colBatchScanBase) GetRowsRead() int64 { s.mu.Lock() defer s.mu.Unlock() return s.mu.rowsRead } -// GetBatchRequestsIssued is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetBatchRequestsIssued() int64 { - s.mu.Lock() - defer s.mu.Unlock() - return s.cf.getBatchRequestsIssued() -} - // GetContentionInfo is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetContentionInfo() (time.Duration, []roachpb.ContentionEvent) { +func (s *colBatchScanBase) GetContentionInfo() (time.Duration, []roachpb.ContentionEvent) { return execstats.GetCumulativeContentionTime(s.Ctx, nil /* recording */) } // GetScanStats is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetScanStats() execstats.ScanStats { +func (s *colBatchScanBase) GetScanStats() execstats.ScanStats { return execstats.GetScanStats(s.Ctx, nil /* recording */) } -// GetKVCPUTime is part of the colexecop.KVReader interface. -func (s *ColBatchScan) GetKVCPUTime() time.Duration { - s.mu.Lock() - defer s.mu.Unlock() - return s.cf.getKVCPUTime() +// Release implements the execreleasable.Releasable interface. +func (s *colBatchScanBase) Release() { + // Deeply reset the spans so that we don't hold onto the keys of the spans. + s.SpansWithCopy.Reset() + *s = colBatchScanBase{ + SpansWithCopy: s.SpansWithCopy, + } + colBatchScanBasePool.Put(s) } -var colBatchScanPool = sync.Pool{ +func (s *colBatchScanBase) close() error { + if s.tracingSpan != nil { + s.tracingSpan.Finish() + s.tracingSpan = nil + } + return nil +} + +var colBatchScanBasePool = sync.Pool{ New: func() interface{} { - return &ColBatchScan{} + return &colBatchScanBase{} }, } -// NewColBatchScan creates a new ColBatchScan operator. -func NewColBatchScan( +// newColBatchScanBase creates a new colBatchScanBase. +func newColBatchScanBase( ctx context.Context, - fetcherAllocator *colmem.Allocator, kvFetcherMemAcc *mon.BoundAccount, flowCtx *execinfra.FlowCtx, spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, - estimatedRowCount uint64, typeResolver *descs.DistSQLTypeResolver, -) (*ColBatchScan, error) { +) (*colBatchScanBase, *roachpb.BoundedStalenessHeader, *cFetcherTableArgs, error) { // NB: we hit this with a zero NodeID (but !ok) with multi-tenancy. if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); nodeID == 0 && ok { - return nil, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID") + return nil, nil, nil, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID") } var bsHeader *roachpb.BoundedStalenessHeader if aost := flowCtx.EvalCtx.AsOfSystemTime; aost != nil && aost.BoundedStaleness { @@ -221,39 +148,12 @@ func NewColBatchScan( } limitHint := rowinfra.RowLimit(execinfra.LimitHint(spec.LimitHint, post)) - tableArgs, err := populateTableArgs(ctx, &spec.FetchSpec, typeResolver) + tableArgs, err := populateTableArgs(ctx, &spec.FetchSpec, typeResolver, false /* allowUnhydratedEnums */) if err != nil { - return nil, err + return nil, nil, nil, err } - kvFetcher := row.NewKVFetcher( - flowCtx.Txn, - bsHeader, - spec.Reverse, - spec.LockingStrength, - spec.LockingWaitPolicy, - flowCtx.EvalCtx.SessionData().LockTimeout, - kvFetcherMemAcc, - flowCtx.EvalCtx.TestingKnobs.ForceProductionValues, - ) - - fetcher := cFetcherPool.Get().(*cFetcher) - fetcher.cFetcherArgs = cFetcherArgs{ - execinfra.GetWorkMemLimit(flowCtx), - estimatedRowCount, - flowCtx.TraceKV, - true, /* singleUse */ - execstats.ShouldCollectStats(ctx, flowCtx.CollectStats), - } - - if err = fetcher.Init( - fetcherAllocator, kvFetcher, tableArgs, - ); err != nil { - fetcher.Release() - return nil, err - } - - s := colBatchScanPool.Get().(*ColBatchScan) + s := colBatchScanBasePool.Get().(*colBatchScanBase) s.Spans = spec.Spans if !flowCtx.Local { // Make a copy of the spans so that we could get the misplanned ranges @@ -264,7 +164,7 @@ func NewColBatchScan( // any other component), but we can use the memory account of the KV // fetcher. if err = kvFetcherMemAcc.Grow(ctx, s.Spans.MemUsage()); err != nil { - return nil, err + return nil, nil, nil, err } s.MakeSpansCopy() } @@ -282,28 +182,108 @@ func NewColBatchScan( } } - *s = ColBatchScan{ + *s = colBatchScanBase{ SpansWithCopy: s.SpansWithCopy, flowCtx: flowCtx, - bsHeader: bsHeader, - cf: fetcher, limitHint: limitHint, batchBytesLimit: batchBytesLimit, parallelize: spec.Parallelize, - ResultTypes: tableArgs.typs, } - return s, nil + return s, bsHeader, tableArgs, nil +} + +// ColBatchScan is the colexecop.Operator implementation of TableReader. It +// reads a table from the KV layer, presenting it as coldata.Batches via the +// colexecop.Operator interface. +type ColBatchScan struct { + *colBatchScanBase + cf *cFetcher +} + +// ScanOperator combines common interfaces between operators that perform KV +// scans, such as ColBatchScan and ColIndexJoin. +type ScanOperator interface { + colexecop.KVReader + execreleasable.Releasable + colexecop.ClosableOperator +} + +var _ ScanOperator = &ColBatchScan{} + +// Init initializes a ColBatchScan. +func (s *ColBatchScan) Init(ctx context.Context) { + if !s.InitHelper.Init(ctx) { + return + } + // If tracing is enabled, we need to start a child span so that the only + // contention events present in the recording would be because of this + // cFetcher. Note that ProcessorSpan method itself will check whether + // tracing is enabled. + s.Ctx, s.tracingSpan = execinfra.ProcessorSpan(s.Ctx, "colbatchscan") + limitBatches := !s.parallelize + if err := s.cf.StartScan( + s.Ctx, + s.Spans, + limitBatches, + s.batchBytesLimit, + s.limitHint, + ); err != nil { + colexecerror.InternalError(err) + } +} + +// Next is part of the colexecop.Operator interface. +func (s *ColBatchScan) Next() coldata.Batch { + bat, err := s.cf.NextBatch(s.Ctx) + if err != nil { + colexecerror.InternalError(err) + } + if bat.Selection() != nil { + colexecerror.InternalError(errors.AssertionFailedf("unexpectedly a selection vector is set on the batch coming from CFetcher")) + } + s.mu.Lock() + s.mu.rowsRead += int64(bat.Length()) + s.mu.Unlock() + return bat +} + +// DrainMeta is part of the colexecop.MetadataSource interface. +func (s *ColBatchScan) DrainMeta() []execinfrapb.ProducerMetadata { + trailingMeta := s.colBatchScanBase.drainMeta() + meta := execinfrapb.GetProducerMeta() + meta.Metrics = execinfrapb.GetMetricsMeta() + meta.Metrics.BytesRead = s.GetBytesRead() + meta.Metrics.RowsRead = s.GetRowsRead() + trailingMeta = append(trailingMeta, *meta) + return trailingMeta +} + +// GetBytesRead is part of the colexecop.KVReader interface. +func (s *ColBatchScan) GetBytesRead() int64 { + s.mu.Lock() + defer s.mu.Unlock() + return s.cf.getBytesRead() +} + +// GetBatchRequestsIssued is part of the colexecop.KVReader interface. +func (s *ColBatchScan) GetBatchRequestsIssued() int64 { + s.mu.Lock() + defer s.mu.Unlock() + return s.cf.getBatchRequestsIssued() +} + +// GetKVCPUTime is part of the colexecop.KVReader interface. +func (s *ColBatchScan) GetKVCPUTime() time.Duration { + s.mu.Lock() + defer s.mu.Unlock() + return s.cf.getKVCPUTime() } -// Release implements the execinfra.Releasable interface. +// Release implements the execreleasable.Releasable interface. func (s *ColBatchScan) Release() { + s.colBatchScanBase.Release() s.cf.Release() - // Deeply reset the spans so that we don't hold onto the keys of the spans. - s.SpansWithCopy.Reset() - *s = ColBatchScan{ - SpansWithCopy: s.SpansWithCopy, - } - colBatchScanPool.Put(s) + *s = ColBatchScan{} } // Close implements the colexecop.Closer interface. @@ -313,9 +293,54 @@ func (s *ColBatchScan) Close(context.Context) error { // span. ctx := s.EnsureCtx() s.cf.Close(ctx) - if s.tracingSpan != nil { - s.tracingSpan.Finish() - s.tracingSpan = nil + return s.colBatchScanBase.close() +} + +// NewColBatchScan creates a new ColBatchScan operator. +// +// It also returns a slice of resulting column types from this operator. It +// should be used rather than the slice of column types from the scanned table +// because the scan might synthesize additional implicit system columns. +func NewColBatchScan( + ctx context.Context, + fetcherAllocator *colmem.Allocator, + kvFetcherMemAcc *mon.BoundAccount, + flowCtx *execinfra.FlowCtx, + spec *execinfrapb.TableReaderSpec, + post *execinfrapb.PostProcessSpec, + estimatedRowCount uint64, + typeResolver *descs.DistSQLTypeResolver, +) (*ColBatchScan, []*types.T, error) { + base, bsHeader, tableArgs, err := newColBatchScanBase( + ctx, kvFetcherMemAcc, flowCtx, spec, post, typeResolver, + ) + if err != nil { + return nil, nil, err } - return nil + kvFetcher := row.NewKVFetcher( + flowCtx.Txn, + bsHeader, + spec.Reverse, + spec.LockingStrength, + spec.LockingWaitPolicy, + flowCtx.EvalCtx.SessionData().LockTimeout, + kvFetcherMemAcc, + flowCtx.EvalCtx.TestingKnobs.ForceProductionValues, + ) + fetcher := cFetcherPool.Get().(*cFetcher) + fetcher.cFetcherArgs = cFetcherArgs{ + execinfra.GetWorkMemLimit(flowCtx), + estimatedRowCount, + flowCtx.TraceKV, + true, /* singleUse */ + execstats.ShouldCollectStats(ctx, flowCtx.CollectStats), + } + if err = fetcher.Init(fetcherAllocator, kvFetcher, tableArgs); err != nil { + fetcher.Release() + return nil, nil, err + } + return &ColBatchScan{ + colBatchScanBase: base, + cf: fetcher, + }, tableArgs.typs, nil } diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index b97025f225cf..0f1faf3f3abb 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -504,7 +504,7 @@ func NewColIndexJoin( return nil, errors.AssertionFailedf("non-empty ON expressions are not supported for index joins") } - tableArgs, err := populateTableArgs(ctx, &spec.FetchSpec, typeResolver) + tableArgs, err := populateTableArgs(ctx, &spec.FetchSpec, typeResolver, false /* allowUnhydratedEnums */) if err != nil { return nil, err } diff --git a/pkg/sql/colfetcher/vectorized_batch_size_test.go b/pkg/sql/colfetcher/vectorized_batch_size_test.go index 568115165efb..86415e9fa699 100644 --- a/pkg/sql/colfetcher/vectorized_batch_size_test.go +++ b/pkg/sql/colfetcher/vectorized_batch_size_test.go @@ -91,9 +91,15 @@ func TestScanBatchSize(t *testing.T) { conn := tc.Conns[0] sqlDB := sqlutils.MakeSQLRunner(conn) + // Until we propagate the estimated row count hint in the KV projection + // pushdown case, this test is expected to fail if the direct scans are + // used (#94850). + _, err := conn.ExecContext(ctx, `SET direct_columnar_scans_enabled = false`) + assert.NoError(t, err) + // Disable automatic table stats collection so that we can control whether // they are present or not. - _, err := conn.ExecContext(ctx, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled=false;`) + _, err = conn.ExecContext(ctx, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled=false;`) assert.NoError(t, err) for _, testCase := range scanBatchSizeTestCases { t.Run(testCase.query, func(t *testing.T) { @@ -156,9 +162,15 @@ func TestCFetcherLimitsOutputBatch(t *testing.T) { defer tc.Stopper().Stop(ctx) conn := tc.Conns[0] + // Until we propagate the estimated row count hint in the KV projection + // pushdown case, this test is expected to fail if the direct scans are + // used (#94850). + _, err := conn.ExecContext(ctx, `SET direct_columnar_scans_enabled = false`) + assert.NoError(t, err) + // Lower the distsql_workmem session variable to 128KiB to speed up the // test. - _, err := conn.ExecContext(ctx, `SET distsql_workmem='128KiB';`) + _, err = conn.ExecContext(ctx, `SET distsql_workmem='128KiB';`) assert.NoError(t, err) for _, tc := range []struct { diff --git a/pkg/sql/colflow/vectorized_flow_planning_test.go b/pkg/sql/colflow/vectorized_flow_planning_test.go index 958f93c7739a..cfd900838c61 100644 --- a/pkg/sql/colflow/vectorized_flow_planning_test.go +++ b/pkg/sql/colflow/vectorized_flow_planning_test.go @@ -40,10 +40,14 @@ func TestVectorizedPlanning(t *testing.T) { // true; if it isn't, we skip this test. return } + // Disable the direct columnar scans to make the output below + // deterministic. + _, err := conn.ExecContext(ctx, `SET direct_columnar_scans_enabled = false`) + require.NoError(t, err) // Check that there is no columnarizer-materializer pair on top of the // root of the execution tree if the root is a wrapped row-execution // processor. - _, err := conn.ExecContext(ctx, `CREATE TABLE t (id INT PRIMARY KEY)`) + _, err = conn.ExecContext(ctx, `CREATE TABLE t (id INT PRIMARY KEY)`) require.NoError(t, err) rows, err := conn.QueryContext(ctx, `EXPLAIN (VEC, VERBOSE) SELECT * FROM t AS t1 INNER LOOKUP JOIN t AS t2 ON t1.id = t2.id`) require.NoError(t, err) diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 7af816dc5bdd..cc878fd16d09 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -3040,6 +3040,10 @@ func (m *sessionDataMutator) SetSynchronousCommit(val bool) { m.data.SynchronousCommit = val } +func (m *sessionDataMutator) SetDirectColumnarScansEnabled(b bool) { + m.data.DirectColumnarScansEnabled = b +} + func (m *sessionDataMutator) SetDisablePlanGists(val bool) { m.data.DisablePlanGists = val } diff --git a/pkg/sql/explain_bundle.go b/pkg/sql/explain_bundle.go index 80910bb7e4e2..17cf5110b111 100644 --- a/pkg/sql/explain_bundle.go +++ b/pkg/sql/explain_bundle.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/colfetcher" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" "github.com/cockroachdb/cockroach/pkg/sql/opt/memo" @@ -29,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/duration" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/memzipper" @@ -676,6 +678,7 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer, sv *settings.Values {sessionSetting: "default_transaction_quality_of_service"}, {sessionSetting: "default_transaction_read_only"}, {sessionSetting: "default_transaction_use_follower_reads"}, + {sessionSetting: "direct_columnar_scans_enabled", clusterSetting: colfetcher.DirectScansEnabled, convFunc: boolToOnOff}, {sessionSetting: "disallow_full_table_scans", clusterSetting: disallowFullTableScans, convFunc: boolToOnOff}, {sessionSetting: "distsql", clusterSetting: DistSQLClusterExecMode, convFunc: distsqlConv}, {sessionSetting: "enable_implicit_select_for_update", clusterSetting: implicitSelectForUpdateClusterMode, convFunc: boolToOnOff}, @@ -722,6 +725,14 @@ func (c *stmtEnvCollector) PrintSessionSettings(w io.Writer, sv *settings.Values } } else { def = s.clusterSetting.EncodedDefault() + if buildutil.CrdbTestBuild { + // In test builds we might randomize some setting defaults, so + // we need to override them to make the tests deterministic. + switch s.sessionSetting { + case "direct_columnar_scans_enabled": + def = "false" + } + } } if s.convFunc != nil { // If necessary, convert the encoded cluster setting to a session setting diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 4bf6bdf89845..37f62e7105cb 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -1692,6 +1692,14 @@ func (t *logicTest) newCluster( } t.outf("setting distsql_workmem='%dB';", randomWorkmem) } + + if serverArgs.DisableDirectColumnarScans { + if _, err := conn.Exec( + "SET CLUSTER SETTING sql.distsql.direct_columnar_scans.enabled = false", + ); err != nil { + t.Fatal(err) + } + } } if cfg.OverrideDistSQLMode != "" { @@ -4062,6 +4070,8 @@ type TestServerArgs struct { // If positive, it provides a lower bound for the default-batch-bytes-limit // metamorphic constant. BatchBytesLimitLowerBound int64 + // If set, sql.distsql.direct_columnar_scans.enabled is set to false. + DisableDirectColumnarScans bool } // RunLogicTests runs logic tests for all files matching the given glob. diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index 7d12b07a684e..0466862f5247 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -96,6 +96,11 @@ SELECT 1::INT2, s COLLATE en FROM t66306; ---- 1 foo +# Always enable the direct columnar scans to make the output below +# deterministic. +statement ok +SET direct_columnar_scans_enabled = true + # Sanity check that the wrapped processor is planned for the query above. If it # no longer is, we should adjust the query here and above. query T @@ -108,4 +113,7 @@ EXPLAIN (VEC) SELECT 1::INT2, s COLLATE en FROM t66306; └ *colrpc.Outbox └ *colexecbase.castIntInt2Op └ *rowexec.noopProcessor - └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchDirectScan + +statement ok +RESET direct_columnar_scans_enabled diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_enum b/pkg/sql/logictest/testdata/logic_test/distsql_enum index 51a5025bc4df..679daaebf7b7 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_enum +++ b/pkg/sql/logictest/testdata/logic_test/distsql_enum @@ -62,6 +62,11 @@ start_key end_key replicas lease_holder …/1/0 {3} 3 …/1/20 {3} 3 +# Always enable the direct columnar scans to make the output below +# deterministic. +statement ok +SET direct_columnar_scans_enabled = true + # Ensure that the join readers are planned on the remote nodes. query T EXPLAIN (VEC) @@ -71,19 +76,22 @@ SELECT t1.x from t1 INNER LOOKUP JOIN t2 ON t1.x=t2.x WHERE t2.y='hello' ├ Node 1 │ └ *colexec.ParallelUnorderedSynchronizer │ ├ *rowexec.joinReader -│ │ └ *colfetcher.ColBatchScan +│ │ └ *colfetcher.ColBatchDirectScan │ ├ *colrpc.Inbox │ └ *colrpc.Inbox ├ Node 2 │ └ *colrpc.Outbox │ └ *rowexec.joinReader -│ └ *colfetcher.ColBatchScan +│ └ *colfetcher.ColBatchDirectScan └ Node 3 └ *colrpc.Outbox └ *rowexec.joinReader - └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchDirectScan query I SELECT t1.x from t1 INNER LOOKUP JOIN t2 ON t1.x=t2.x WHERE t2.y='hello' ---- 0 + +statement ok +RESET direct_columnar_scans_enabled diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index b71ba9af19f8..bb6b180a32d0 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -4769,7 +4769,8 @@ WHERE 'experimental_distsql_planning', 'use_declarative_schema_changer', 'distsql_workmem', - 'copy_fast_path_enabled' + 'copy_fast_path_enabled', + 'direct_columnar_scans_enabled' ); ---- variable value diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 8d0e7c822ede..884252c2e9d9 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -2526,7 +2526,7 @@ SELECT FROM pg_catalog.pg_settings WHERE - name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled') + name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled') ---- name setting category short_desc extra_desc vartype allow_ordinal_column_references off NULL NULL NULL string @@ -2670,7 +2670,7 @@ SELECT FROM pg_catalog.pg_settings WHERE - name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled') + name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled') ---- name setting unit context enumvals boot_val reset_val allow_ordinal_column_references off NULL user NULL off off @@ -2837,6 +2837,7 @@ default_transaction_read_only NULL NULL NULL default_transaction_use_follower_reads NULL NULL NULL NULL NULL default_with_oids NULL NULL NULL NULL NULL descriptor_validation NULL NULL NULL NULL NULL +direct_columnar_scans_enabled NULL NULL NULL NULL NULL disable_hoist_projection_in_join_limitation NULL NULL NULL NULL NULL disable_partially_distributed_plans NULL NULL NULL NULL NULL disable_plan_gists NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index b45018f0f480..786db78d8fb6 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -23,7 +23,7 @@ UTF8 1 query TT colnames SELECT * FROM [SHOW ALL] -WHERE variable NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled') +WHERE variable NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled') ---- variable value allow_ordinal_column_references off diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain_env b/pkg/sql/opt/exec/execbuilder/testdata/explain_env index 3869c90833cf..201c5906f7f0 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain_env +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain_env @@ -39,7 +39,7 @@ CREATE TABLE y ( query T EXPLAIN (OPT, ENV) SELECT * FROM x WHERE b = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt9O47gXvp48xVFvBn5qIaW_C1Q00nZKVtvdUlCbmR2EkOUkJ6kXxza2UwqrleYheEKeZGWnQGdFmh2tphdV457vO_--c5xeDz6jNkyKIYxleqMlTZenHwHXmCYV4xlqsGgsrGqrIOj1gHIu74jSqKhGQg2RyhLFqYAlNWCXCBnmtOIWVpRXOASZ5w6XSmOJSakw5I7ZJdlYkVRyYtgDtsAz6iK55412k8V5F85OL73xhpsJu5P7eNvYaioMTS2TgijNpGb2vgkopC4pb0LfVpQze09kTgzqFUsbA9BYVJzqJiKNNCNS8MY4novzBrYySHLpmoXaE5k2Embq3uYV58TShGPdr124RRQ7oDW3HD64kxOA14CeLWllpXOBwrOyUnGWMksMckwtyaUmlXIdbnQl3oJvp-s4EmrTJTGWWixR2ObAv2ETBrUlOTWWKGqX_wpUVtwy5X7IjOUspS4G4xru69ZS6Q3JAyseaEH-kKx5dmqna-XKREVGWCGkRiKkJStmmKNJJa9KYQgTJJWqTSpMWNQrynfOkpLGFhqNA3CqC6w14dRAtLxrLGw_DEOPkelmApRlJXvAjCiqLXNVwowwkeHas-3OWzinUmeoMSOcGtuSm1O9lxHRuJQlulhJXexstye3w4zirN5QhLOSNTo7Cv9_vMH43PRm1DSm1LSJ7lvQkhkrC03L70J58fm1aen3-duWjW8Cti0FVRCri_LAsJJx6lYisUuNZil5Y0nDg4GHasxREy7lTaW8yo2f0vym1amWihauj0yoytYSYKJogWn0hhtXO3voG2huOTE0x41o2qJy9w8TBXktqb_SFGpb6cRvgMaKvI13G9e1QlccidIyoQnjO26dBhpNRSZLYrBZ5TWSlfggRePUf4rHzqwSqRTGasrcrArp1srK139rcFvHqq7ZClMr9a7bXQTBeB6N4gji0cdpBKpKOEsP1rAXvKMwmcXHMDuPYfZpOu0G75LNSf00Pp8t4vloMothTdQN3sPFfHI2ml_Cb9El7FEYLcb73eDdZHYafYE1SQjL1rCX-PNg_yQIRtM4mv_T82T2azSOYRGP4skinowX8P4qAAD403-7T4euCv9m0RlC2H093uzizhCuXg5r-87L8_W2vUZqMSPUdobQOQr7x72w3wv7EPaHYTgMw86WsbtnmUjdK1MlHKAfbvv2y8S_Ttl75QLrbIP9Ln0GbsPcjnwhPBr0jwb-v7-6_zXl5Iek7CP8cVkH1-9PgiD6cjEdTWawd34RdyGafd6HRTR1ovgf_Dw_P4M1_P5LNI8ggQ8wOAl6vV4v8PfZ-qeNzAJ4enx8evz69PgVXubJDuHw6LA_hKvDAfTgcHAd_B0AAP__4rTYRg== +https://cockroachdb.github.io/text/decode.html#eJy0VuFu2zYQ_l09xcF_mg52I8f7ETgoMNfVMG-uU9hq1yAICEo6yVwokiEpx8kwoA-RJ8yTDKSUxB0iq8VQ_zAs-r67491332kwgE-oDZNiDFOZXmpJ0_W7t4BbTJOK8Qw1WDQWNrVVEAwGQDmX10RpVFQjoYZIZYniVMCaGrBrhAxzWnELG8orHIPMc4dLpbHEpFQYcs3smjRWJJWcGHaLHfCMukxueKvdbHXah_fvzrxx45sJu9f38a6x1VQYmlomBVGaSc3sTRtQSF1S3oa-qihn9obInBjUG5a2JqCxqDjVbY400oxIwVvzeCjOM9jKIMmlaxZq78h0OWEaU9-PqhRUN71CQROOWSfW1LzIK86JdZAavw-3imIHtOaKwxt3cgLwdJkHS1pZ6ULUiRBWKs5SZolB7tLNpSaVcuxoDSWeg--WyvlIqE3XxFhqsURh2xP_ypswqC3JqbFEUbv-JlBZccuU-yEzlrOUuhyMI4uvW0elGye3rLilBflLsva5q4NulSsTFRlhhZAaiZCWbJhhzk3dbEOYIKlUXTRjwqLeUL53DpU0ttBoHIBTXWDNCccGouV1a2GHYRh6jEyb6VGWlewWM6KotsxVCTPCRIZb723_vYULKnWGGjPCqbEdd3MT42lENK5liS7XTvL7SE7_jOKsVjfCWclagx2FPx83GH833YypxpSaLtJ9DVozY2WhafldKE8-L7mWfl-8Xdr4JmCXoKiCWF2Urw0rGadOToldazRryVtLGr4eeajGHDXhUl5WyrPc-CnNLzuDaqlo4frIhKpsTQEmig6YRm_YhNrbQ99Ac8WJoTk2pOnKyu0uJgryVFK_DhVqW-nEK0BrRZ7HO8V1rdAVR6K0TGjC-J6N1eJGU5HJkhhsZ3mNZCXeStE69R_jqTOrRCqFsZoyN6tCOlnZ-PrvDO437pQNplbqfW8GIgimy2gSRxBP3s4jUFXCWfp6CwfBCwqzRXwMi9MYFh_n837wImlO6qfp6WIVLyezRQxboi7xBj4sZ-8nyzP4IzqDAwqT1fRVP3gxW7yLPsOWJIRlWzhI_Hnw6iQIJvM4Wv438mzxezSNYRVP4tkqnk1X8PI8AAD423-7T49uCv9W0htD2H86brS4N4bzx8Pavvf4fLFrr5FazAi1vTH0jsLh8SAcDsIhhMNxGI7DsLdj7PYsE369V8IBhuFubC8m_lXM3iiXWG8X7LX0AbgLcxr56PBoNDwa-f_-6f_fKyc_5Mo-wx936-Di5UkQRJ8_zCezBRycfoj7EC0-vYJVNHek-Al-XZ6-hy38-Vu0jCCBNzA6CQaDwSDw-2z7S0OzAO7v7u7vvtzffYHHebJjODw6HI7h_HAEAzgcXQT_BgAA__9Fnu46 statement error ENV only supported with \(OPT\) option @@ -52,7 +52,7 @@ EXPLAIN (ENV) SELECT * FROM x WHERE b = 3 query T EXPLAIN (OPT, ENV) SELECT * FROM x, y WHERE b = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt9u27YXvq6e4sA3dX6wGzn5XQQOCsx11c1b6hS22rUoCoKSjmQuFMmSlGpnGFDsGXK5p8uTDKSUxF0te8XQXAQ2_X3n73cOORzCG9SGSTGGqUyvtKTp6vkzwDWmScV4hhosGgt1gwqC4RAo5_ITURoV1UioIVJZojgVsKIG7Aohw5xW3EJNeYVjkHnueKk0lpiUCkM-MbsiLYqkkhPDrvEAPaMukg3vxM2WlwN4-fydB7e2mbB7bZ9tg62mwtDUMimI0kxqZjddRCF1SXkX-2NFObMbInNiUNcs7QxAY1FxqrsMaaQZkYJ3xnFXnB3cyiDJpWsWam_IHDLCTNPbvOKcWJpwbPq1j7eMYke05iOHp-7kHOAhoDskrax0LlB4q6xUnKXMEoMcU0tyqUmlXIc7XYld9O10nY2E2nRFjKUWSxS2O_AvrAmD2pKcGksUtat_RSorbplyH2TGcpZSF4NxDfd1O1Dp1sg1K65pQX6TrHt2Gqdr5cpERUZYIaRGIqQlNTPMmUklr0phCBMkleqQVJiwqGvK986SksYWGo0jcKoLbDTh1EC0_NRZ2FEYhp4j03YClGUlu8aMKKotc1XCjDCR4dpb25-3cE6lzlBjRjg19kBuTvVeRkTjSpboYiVNsbP9ntwOM4qzZkMRzkrW6ewk_P9Zy_G56XbUNKbUHBLdl6QVM1YWmpbfxPLi82vT0m_zty0b3wQ8tBRUQawuyieGlYxTtxKJXWk0K8k7Sxo-OfVUjTlqwqW8qpRXufFTml8ddKqlooXrIxOqso0EmCgO0DR6YOtqbw99A81HTgzNsRXNoajc_cNEQR5K6q80hdpWOvEboLMiu_lu47pW6IojUVomNGF8z63TYUZTkcmSGOxWecNkJV5L0Tn1r-Opg1UilcJYTZmbVSHdWql9_bcG9-BYNTWrMbVS77vdRRBMF9EkjiCePLuIQFUJZ-mTNfSDRxRm8_gM5pcxzF9fXAyCR0l70nybXs6X8WIym8ewJuoKN_BqMXs5WbyDX6J30KcwWU6PBsGj2fx59BbWJCEsW0M_8efB0XkQTC7iaPFPz7P5z9E0hmU8iWfLeDZdwuP3AQDA7_6_--vRuvAvi94YwsHDcbuLe2N4f3_Y4Hv33z9s4zVSixmhtjeG3kk4OhuGo2E4gnA0DsNxGPa2wO6eZSJ1T6ZKOMIo3Pbtl4l_TtmNcoH1tsl-l94Rt2luR94bPDkdnZz63_4Y_NeUk--Sso_w-2UdfHh8vluRG6fI6itF1l2K3OxQZHWnyC9wNckd8sXlIpr9OG-Q9REsohfRIppPo-W9NPv0Qc6O5-Vc75XzZqecfZbR21cXk9kc-pev4gFE8zdHsIwuHPZ_8GJx-RLWA9jArz9FiwgSeAqn58FwOBwGTAjUQ_9s6adaGnMUwO3NX7c3n29vPoO_1jdfnax_aOfP_fKnq_XtzU0LuN83dgzHJ8ejMbw_PoUhHJ9-CLZgOeMWtYG-1RUeBX8HAAD__wttO5A= +https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvq6e4sA3dQa7kZNdBA4KzHXVzVvqFLbatSgKgpaOZC4UqR5Srp1hQLFnyOWeLk8ykFISZ43sFkNzEdj0953f7xyy34c3SEZoNYSxTi5I82T5_BngGpNFJWSKBBaNhVWNCoJ-H7iU-hMrCUtOyLhhurSslFzBkhuwS4QUM15JCysuKxyCzjLHS7SxzCRcGfZJ2CVrUCzRkhlxiXvoKXeRbGQrbjI_78HL5-88uLEtlN1p-2QbbIkrwxMrtGIlCU3CbtqISlPBZRv7Y8WlsBumM2aQViJpDYAwrySnNkOEPGVaydY4borzALcyyDLtmoXkDZl9RgRh4vtRFYpT0ytUfCEx3cs1tS6ySkpmHaXm7-LNo9gRrfko4ak7OQW4S-YGySurnYs6ECaKUopEWGZQunAzTawqnTpaXamH6NulcjYW3CZLZiy3WKCy7YHfs6YMkmUZN5aV3C6_ilRU0orSfdCpyETCXQzGicXXbU-lGyOXIr_kOftDi_a5q52uS1cmrlImcqUJmdKWrYQRzkzdbMOEYoku98lMKIu04nLnHJba2JzQOILklGOtCacGRvpTa2EHYRh6jk6a6SmtKMQlpqzkZIWrEqZMqBTX3truvJVzqilFwpRJbuye3NzEeBkxwqUu0MW6V_zek9t_ppSi3m5MikK0OjsKfzxpOD43asaUMOFmn-juk5bCWJ0TL76J5cXnV67l3-ZvWza-CbhvoZQ5s5QXT4wohORunTK7JDRLLVtLGj459lTCDIlJrS-q0qvc-CnNLvY6JV3y3PVRqLKytQSEyvfQCD2wcbWzh76B5qNkhmfYiGZfVO7uEipndyX112GJZCta-A3QWpGH-W7julZQJZGVpBd8IeSOG6vFDHGV6oIZbFd5zRQFXmrVOvWv47GDVSrRyljiws2q0m6trHz9twb3K--UFSZW066XgQqC8SwaxRHEo2dnEZTVQorkyRq6wSMOk2l8AtPzGKavz856waNFc1J_G59P5_FsNJnGsGblBW7g1WzycjR7B79F76DLYTQfH_SCR5Pp8-gtrNmCiXQN3YU_Dw5Og2B0Fkez_3qeTH-NxjHM41E8mceT8Rwevw8AAP70_91fh69y_yrpDCHs3R03u7gzhPe3hzW-c_v9wzaekFtMGbedIXSOwsFJPxz0wwGEg2EYDsOwswV296xQ_nqvlCMMwm3ffpn4p5jdlC6wzjbZ79Ib4jbN7chbg0fHg6Nj_9tfvf-b8uK7pOwj_H5ZBx8enz6syI1TZPWFIldtitw8oMjqRpH3cCuWOeSL81k0-XlaI1cHMIteRLNoOo7mt9Ls8js5O56X82qnnDcPytlnGb19dTaaTKF7_iruQTR9cwDz6Mxhf4AXs_OXsO7BBn7_JZpFsICncHwa9Pv9fiCUQur7Z0s3IW3MQQDXV_9cX32-vvoM_lrffHGy_qmZP_fL367W11dXDeB239ghHB4dDobw_vAY-nB4_CHYgmVCWiQDXUsVHgT_BgAA__9dJVGE # # Same table twice should only show up once. @@ -61,7 +61,7 @@ https://cockroachdb.github.io/text/decode.html#eJy0Vt9u27YXvq6e4sA3dX6wGzn5XQQOC query T EXPLAIN (OPT, ENV) SELECT * FROM x one, x two ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vs9u2zgTP1dPMfClzge7kJPvEDjowXW8gHcdJ7DVokEQELQ0krmhSIYcOU4WC_Qhctyny5MsSDmJu4jsLRb1wZDI-c3f38yo24UvaJ3Qqg9Dnd5YzdPl6SfANaaLSsgMLRA6glUtFUXdLnAp9R0zFg23yLhj2hAzkitYcge0RMgw55UkWHFZYR90nntcqh0xl3Ll2J2gJdtIsVRL5sQD7oFn3HtyLxvlxvPzDpydXgbhjW6haKfu421hslw5npLQihkrtBV03wRU2pZcNqFvKy4F3TOdM4d2JdJGBywWleS2SZFFnjGtZKMfz8l5A1s5ZLn2xUIbFLl9SoSra5tXUjLiC4l1vXbh5qPEA8ndSvjoT04AXh16luQVaW8CVdAqSiNFKog5lJgSy7VllfEVbjSl3oJvh-t1LDilS-aIE5aoqNnx77Qph5ZYzh0xw2n5r0BlJUkY_6AzkYuUex-cL3jI255Mb5Q8iOKBF-x3LZp7pza6Nj5NXGVMFEpbZEoTWwknvJpUy6pUjgnFUm32UUUoQrvicmcvGe2osOg8QHJbYM0JzwZm9V1jYntxHAeMTjcdYEiU4gEzZrgl4bOEGRMqw3XQtjtu5Y1qm6HFjEnuaE9snvWBRsziUpfofWV1srPdlvwMc0aKekIxKUrRaOww_v_xBhNis5tWs5hyt49034OWwpEuLC9_CBXIF8Ym8R-zt02bUATcNxRMwcgW5QcnSiG5H4mMlhbdUsvGlMYfjgLUYo6WSa1vKhNY7kKX5jd7jVpteOHrKJSpqKaAUMUemMUguDG1s4ahgO5WMsdz3JBmn1d-_whVsNeUhpVm0FJlF2ECNGbkbbyfuL4UtpLIjNULvhByx9ZpUGO5ynTJHDazvEaKEh-0auz6z8nQi1Uq1cqR5cL3qtJ-rKxC_rcad29b1TlbYUra7truKoqGs9EgGUEy-DQZgakWUqQf1tCO3nEYT5NjmJ4nMP08mXSid4vNSf02PJ_Ok9lgPE1gzcwN3sPFbHw2mF3Cb6NLaHMYzIcHnejdeHo6-gprtmAiW0N7Ec6jg5MoGkyS0eyflsfTX0fDBObJIBnPk_FwDu-vIgCAP8K__7X4qghfFq0-xJ3X480sbvXh6uWwlm-9vF9vy1vkhBnj1OpD6zDuHXfjXjfuQdzrx3E_jltbwn7PCpX6T6ZKeUAv3rYdhkn4nKJ74x1rbYPDLH0GbsP8jHxReHjUOzwKd392_mvIi58ScvDw50UdXb8_iaLR14vJYDyF9vlF0oHR9MsBzEcTT4r_wS-z8zNYw2AOWmGnfqI7fRJ1u91uJJRC2w0rvZ1a7dxBBE-Pfz09fnt6_AZh5a3hiruPWuF1wxXd6XD1uLnKhSS0DtpkKzyI_g4AAP__FRvwug== +https://cockroachdb.github.io/text/decode.html#eJy0Vs1u2zgQPldPMfClzsIu5HQPgYMeXNcLeNd1ClstWhQFQUsjmRuKZIeU42SxQB8ix326PsmClJK4i8hqsagPhkTON7_fzGg4hHdIVmg1hqlOL0nzdPvqJeAe000lZIYEDq2DXS0VRcMhcCn1FTOEhhMybpk2jhnJFWy5BbdFyDDnlXSw47LCMeg897hUW8dsypVlV8JtWSPFUi2ZFTfYAc-49-RatsrN1xcDeP3qQxBudAvljuo-OxR2xJXlqRNaMUNCk3DXbUClqeSyDf254lK4a6ZzZpF2Im11gLCoJKc2RYQ8Y1rJVj_ukvMItrLIcu2LhRQU2S4lgjAN9ahKxampFSq-kZh1Ym3Ni7ySkjkPqfHHcOtZ4oHOfpbwwp-cAzwEcyfJK6e9idoRJkojRSocsyi9u7kmVhnPjlZT6jH4Yaq8jg136ZZZxx2WqFy7499oUxbJsZxbxwx32-8ClZV0wvgHnYlcpNz7YD1ZQt46Mt0ouRHFDS_Yn1q0911tdG98mrjKmCiUJmRKO7YTVng1dbEtE4ql2nTRTCiHtOPyaB8abV1BaD1Aciqw5oRnAyN91ZrYURzHAaPTpnuME6W4wYwZTk74LGHGhMpwH7Qdj1t5o5oyJMyY5NZ1xOY7JtCIEW51id7XTvIHS37-WSNFPd2YFKVoNXYa_3rWYEJs1LQpYcptF-m-BW2FdbogXv4QKpAvjFzHf8zeIW1CEbBroJiCOSrKZ1aUQnI_TpnbEtqtlq0pjZ89D1DCHIlJrS8rE1huQ5fml51GSRte-DoKZSpXU0CoogNGGAQbU0drGApoP0tmeY4Nabq88rtLqII9pDSsQ4PkKtqECdCakcfxfuL6UlAlkRnSG74R8sjGalFDXGW6ZBbbWV4jRYk3WrV2_dtk6sUqlWplHXHhe1VpP1Z2If8HjfudO2WHqdN07MtARdF0NZskM0gmLxczMNVGivTZHvrREw7zZXIGy4sElm8Xi0H0ZNOc1G_Ti-U6WU3mywT2zFziNbxZzV9PVh_gj9kH6HOYrKcng-jJfPlq9h72bMNEtof-JpxHJ-dRNFkks9V_Lc-Xv8-mCayTSTJfJ_PpGp5-jAAA_gr__tfjuyJ8lfTGEA8ejptZ3BvDx_vDWr53__7pUJ6QO8wYd70x9E7j0dkwHg3jEcSjcRyP47h3IOz3rFBhvVfKA0bxoe0wTMKnmLs23rHeITjM0jvgIczPyHuFp89Hp8_D3d-D_xvy5qeEHDz8eVFHn56eR9Hs_ZvFZL6E_sWbZACz5bsTWM8WnhS_wG-ri9ewh8katMJB_eSu9Hk0HA6HkVAKaRhWej8lbe1JBF9v__l6--Xr7RcIK28PH7l9oRV-arlyVzpc3TZXuZAOyULfUYUn0b8BAAD__8HaBr0= # # Set a relevant session variable to a non-default value and ensure it shows up @@ -74,7 +74,7 @@ SET reorder_joins_limit = 63 query T EXPLAIN (OPT, ENV) SELECT * FROM y WHERE u = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvq6e4sA3TQa7leNhCBz0wnXVzZtrF7baNQgCgpaOZC4UyZCUYmcY0IfIE-ZJBlJO4myWvWKoLwyLPt_5_c5HdTrwGbVhUvRhKJMrLWmyfPcWcIXJomQ8RQ0WjYWqtgqCTgco5_KGKI2KaiTUEKksUZwKWFIDdomQYkZLbqGivMQ-yCxzuEQaS0xChSE3zC7JxookkhPDbvEAPKUukzVvtBvNp2348O7cG298M2H3-j7dNraaCkMTy6QgSjOpmV03AYXUBeVN6OuScmbXRGbEoK5Y0piAxrzkVDc50khTIgVvzOOhOTuwpUGSSTcs1N6ROeSEmXq2Wck5sXTBsZ7XPtw8ih3QmmsOb9zJGcBTQg-WtLTShUDhvbJCcZYwSwxyTCzJpCalchNuDCV2wbfLdT4W1CZLYiy1WKCwzYk_8yYMaksyaixR1C7_E6gouWXK_ZApy1hCXQ7GDdz37UCnN05uWX5Lc_KHZM27UwddKdcmKlLCciE1EiEtqZhhzk0ieVkIQ5ggiVSHqMKERV1RvneXlDQ212gcgFOdY80Jxwai5U1jY7thGHqMTDYboCwr2C2mRFFtmesSpoSJFFfe2_66hQsqdYoaU8KpsQdqc6z3NCIal7JAlyupm53uj-Q0zCjOaoUinBWsMdhJ-OPpBuNr05tV05hQc4h0z0FLZqzMNS2-CeXJ52XT0m-Lt00bPwQ8JAoqJ1bnxSvDCsapk0RilxrNUvLGloaveh6qMUNNuJRXpfIsN35Ls6uDQbVUNHdzZEKVtqYAE_khIdLoLTex6iG-gZ96OzXJz9Bcc2JohhveHErMXUFM5OSpq_5WU6htqRdeBBqbshvvRNdNQ5ccidJyQReM77l4GtxoKlJZEIPNRK-RrMBbKRoX_1M8dGalSKQwVlPm1lVIpyyVH8HW7h7crLpnFSZW6n0XvAiC4SwaxBHEg7fjCFS54Cx5tYKj4AWF0SQ-hck0hsmn8bgdvFhsTuqn4XQyj2eD0SSGFVFXuIaPs9GHwewcfovO4YjCYD48bgcvRpN30RdYkQVh6QqOFv48OD4LgsE4jmb_jDya_BoNY5jHg3g0j0fDOby8CAAA_vTf7tOiVe5fLlp9CNtPxxs5bvXh4vGwtm89Pl9u22ukFlNCbasPrZOwe9oJu52wC2G3H4b9MGxtGburlonEvTWVwgG64XZsryf-jcqulUustQ32cvoA3IY5mXx0eNLrnvT8f3-1_2_Ji-9Sss_w-1UdXL48283ItWNk-S9GVk2MXO9gZPnAyGd2Fcmc5fvpLBr9PKktq2OYRe-jWTQZRvNHah7RJzo7nKdztZfO65109lVGXz6OB6MJHE0_xm2IJp-PYR6Nne0P8H42_QBr-P2XaBZBCW-gdxZ0Op1O4C_udQD3d3f3d1_v777Co1rYPrzu9uHidQ868Lp3GfwdAAD__-FUE64= +https://cockroachdb.github.io/text/decode.html#eJy0Vt9u27YXvq6e4sA3TX6wWzn-YQgc9MJ11c2baxe22rUoCoKWjmQuFMmSlGJnGNCHyBPmSQZSTuJsltViqC8Miz7f-fudj-r14D1qw6QYwlgml1rSZP3qJeAGk1XJeIoaLBoLVW0VBL0eUM7lFVEaFdVIqCFSWaI4FbCmBuwaIcWMltxCRXmJQ5BZ5nCJNJaYhApDrphdk50VSSQnhl1jCzylLpMtb7SbLOddePPqozfe-WbCHvV9vm9sNRWGJpZJQZRmUjO7bQIKqQvKm9BfSsqZ3RKZEYO6YkljAhrzklPd5EgjTYkUvDGPu-YcwJYGSSbdsFB7R6bNCdOY-HmUhaB6NysUdMUxbcWamhdZyTmxDlLjj-GWUeyA1nzh8MKdXAA8FHNnSUsrXYg6EcIKxVnCLDHIXbqZ1KRUjh2NocQh-H6rnI8VtcmaGEstFihsc-KPvAmD2pKMGksUtetvAhUlt0y5HzJlGUuoy8E4svi-tXR65-Sa5dc0J39I1rx3ddCNcm2iIiUsF1IjEdKSihnm3NTDNoQJkkjVRjMmLOqK8qN7qKSxuUbjAJzqHGtOODYQLa8aG9sPw9BjZLLbHmVZwa4xJYpqy1yXMCVMpLjx3o7XLVxQqVPUmBJOjW2pzW2MpxHRuJYFulxbye8jOf0zirNa3QhnBWsMdhb-_3yH8bXp3ZpqTKhpI91j0JoZK3NNi-9CefJ5ybX0--Lt08YPAdsEReXE6rx4ZljBOHVySuxao1lL3tjS8NnAQzVmqAmX8rJUnuXGb2l22RpUS0VzN0cmVGlrCjCRtwmRRm-5i1UP8QX8NDioSX6G5gsnhma4401bYu76YiInD131N6JCbUu98iLQ2JTDeCe6bhq65EiUliu6YvzIpdXgRlORyoIYbCZ6jWQFXkvRuPjv4rEzK0UihbGaMreuQjplqfwI9nb3G6-VChMr9bGXAxEE40U0iiOIRy-nEahyxVnybAMnwRMKk1l8DrN5DLN302k3eLLandRP4_lsGS9Gk1kMG6IucQtvF5M3o8VH-C36CCcURsvxaTd4Mpm9ij7AhqwISzdwsvLnwelFEIymcbT4Z-TJ7NdoHMMyHsWTZTwZL-HppwAA4E__7T4dWuX-xaQzhLD7cLyT484QPt0f1vad--fP-_YaqcWUUNsZQucs7J_3wn4v7EPYH4bhMAw7e8buqmXC3_ClcIB-uB_b64l_G7Nb5RLr7IO9nN4B92FOJu8dng36ZwP_31_d_1ry6oeU7DP8cVUHn59eHGbk1jGy_BcjqyZGbg8wsrxj5CO7imTO8vV8EU1-ntWW1SksotfRIpqNo-U9NU_oA50dztO5Okrn7UE6-yqjD2-no8kMTuZv4y5Es_ensIymzvZ_8HoxfwNb-P2XaBFBCS9gcBH0er1e4C_ubQC3Nze3N19vb77CvVrYITzvD-HT8wH04Pngc_B3AAAA__9k5ymi # Make sure it shows up correctly even if it matches the cluster setting. statement ok @@ -84,7 +84,7 @@ SET CLUSTER SETTING sql.defaults.reorder_joins_limit = 63 query T EXPLAIN (OPT, ENV) SELECT * FROM y WHERE u = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvq6e4sA3TQa7leNhCBz0wnXVzZtrF7baNQgCgpaOZC4UyZCUYmcY0IfIE-ZJBlJO4myWvWKoLwyLPt_5_c5HdTrwGbVhUvRhKJMrLWmyfPcWcIXJomQ8RQ0WjYWqtgqCTgco5_KGKI2KaiTUEKksUZwKWFIDdomQYkZLbqGivMQ-yCxzuEQaS0xChSE3zC7JxookkhPDbvEAPKUukzVvtBvNp2348O7cG298M2H3-j7dNraaCkMTy6QgSjOpmV03AYXUBeVN6OuScmbXRGbEoK5Y0piAxrzkVDc50khTIgVvzOOhOTuwpUGSSTcs1N6ROeSEmXq2Wck5sXTBsZ7XPtw8ih3QmmsOb9zJGcBTQg-WtLTShUDhvbJCcZYwSwxyTCzJpCalchNuDCV2wbfLdT4W1CZLYiy1WKCwzYk_8yYMaksyaixR1C7_E6gouWXK_ZApy1hCXQ7GDdz37UCnN05uWX5Lc_KHZM27UwddKdcmKlLCciE1EiEtqZhhzk0ieVkIQ5ggiVSHqMKERV1RvneXlDQ212gcgFOdY80Jxwai5U1jY7thGHqMTDYboCwr2C2mRFFtmesSpoSJFFfe2_66hQsqdYoaU8KpsQdqc6z3NCIal7JAlyupm53uj-Q0zCjOaoUinBWsMdhJ-OPpBuNr05tV05hQc4h0z0FLZqzMNS2-CeXJ52XT0m-Lt00bPwQ8JAoqJ1bnxSvDCsapk0RilxrNUvLGloaveh6qMUNNuJRXpfIsN35Ls6uDQbVUNHdzZEKVtqYAE_khIdLoLTex6iG-gZ96OzXJz9Bcc2JohhveHErMXUFM5OSpq_5WU6htqRdeBBqbshvvRNdNQ5ccidJyQReM77l4GtxoKlJZEIPNRK-RrMBbKRoX_1M8dGalSKQwVlPm1lVIpyyVH8HW7h7crLpnFSZW6n0XvAiC4SwaxBHEg7fjCFS54Cx5tYKj4AWF0SQ-hck0hsmn8bgdvFhsTuqn4XQyj2eD0SSGFVFXuIaPs9GHwewcfovO4YjCYD48bgcvRpN30RdYkQVh6QqOFv48OD4LgsE4jmb_jDya_BoNY5jHg3g0j0fDOby8CAAA_vTf7tOiVe5fLlp9CNtPxxs5bvXh4vGwtm89Pl9u22ukFlNCbasPrZOwe9oJu52wC2G3H4b9MGxtGburlonEvTWVwgG64XZsryf-jcqulUustQ32cvoA3IY5mXx0eNLrnvT8f3-1_2_Ji-9Sss_w-1UdXL48283ItWNk-S9GVk2MXO9gZPnAyGd2Fcmc5fvpLBr9PKktq2OYRe-jWTQZRvNHah7RJzo7nKdztZfO65109lVGXz6OB6MJHE0_xm2IJp-PYR6Nne0P8H42_QBr-P2XaBZBCW-gdxZ0Op1O4C_udQD3d3f3d1_v777Co1rYPrzu9uHidQ868Lp3GfwdAAD__-FUE64= +https://cockroachdb.github.io/text/decode.html#eJy0Vt9u27YXvq6e4sA3TX6wWzn-YQgc9MJ11c2baxe22rUoCoKWjmQuFMmSlGJnGNCHyBPmSQZSTuJsltViqC8Miz7f-fudj-r14D1qw6QYwlgml1rSZP3qJeAGk1XJeIoaLBoLVW0VBL0eUM7lFVEaFdVIqCFSWaI4FbCmBuwaIcWMltxCRXmJQ5BZ5nCJNJaYhApDrphdk50VSSQnhl1jCzylLpMtb7SbLOddePPqozfe-WbCHvV9vm9sNRWGJpZJQZRmUjO7bQIKqQvKm9BfSsqZ3RKZEYO6YkljAhrzklPd5EgjTYkUvDGPu-YcwJYGSSbdsFB7R6bNCdOY-HmUhaB6NysUdMUxbcWamhdZyTmxDlLjj-GWUeyA1nzh8MKdXAA8FHNnSUsrXYg6EcIKxVnCLDHIXbqZ1KRUjh2NocQh-H6rnI8VtcmaGEstFihsc-KPvAmD2pKMGksUtetvAhUlt0y5HzJlGUuoy8E4svi-tXR65-Sa5dc0J39I1rx3ddCNcm2iIiUsF1IjEdKSihnm3NTDNoQJkkjVRjMmLOqK8qN7qKSxuUbjAJzqHGtOODYQLa8aG9sPw9BjZLLbHmVZwa4xJYpqy1yXMCVMpLjx3o7XLVxQqVPUmBJOjW2pzW2MpxHRuJYFulxbye8jOf0zirNa3QhnBWsMdhb-_3yH8bXp3ZpqTKhpI91j0JoZK3NNi-9CefJ5ybX0--Lt08YPAdsEReXE6rx4ZljBOHVySuxao1lL3tjS8NnAQzVmqAmX8rJUnuXGb2l22RpUS0VzN0cmVGlrCjCRtwmRRm-5i1UP8QX8NDioSX6G5gsnhma4401bYu76YiInD131N6JCbUu98iLQ2JTDeCe6bhq65EiUliu6YvzIpdXgRlORyoIYbCZ6jWQFXkvRuPjv4rEzK0UihbGaMreuQjplqfwI9nb3G6-VChMr9bGXAxEE40U0iiOIRy-nEahyxVnybAMnwRMKk1l8DrN5DLN302k3eLLandRP4_lsGS9Gk1kMG6IucQtvF5M3o8VH-C36CCcURsvxaTd4Mpm9ij7AhqwISzdwsvLnwelFEIymcbT4Z-TJ7NdoHMMyHsWTZTwZL-HppwAA4E__7T4dWuX-xaQzhLD7cLyT484QPt0f1vad--fP-_YaqcWUUNsZQucs7J_3wn4v7EPYH4bhMAw7e8buqmXC3_ClcIB-uB_b64l_G7Nb5RLr7IO9nN4B92FOJu8dng36ZwP_31_d_1ry6oeU7DP8cVUHn59eHGbk1jGy_BcjqyZGbg8wsrxj5CO7imTO8vV8EU1-ntWW1SksotfRIpqNo-U9NU_oA50dztO5Okrn7UE6-yqjD2-no8kMTuZv4y5Es_ensIymzvZ_8HoxfwNb-P2XaBFBCS9gcBH0er1e4C_ubQC3Nze3N19vb77CvVrYITzvD-HT8wH04Pngc_B3AAAA__9k5ymi statement ok SET enable_zigzag_join = false @@ -92,7 +92,7 @@ SET enable_zigzag_join = false query T EXPLAIN (OPT, ENV) SELECT * FROM y WHERE u = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvq6e4sA3TQa7leNhCBz0wnXVzZtrF7baNQgCgpaOZC4UyZCUYmcY0IfIE-ZJBlJO4myWvWKoLwyL_r7z-51DdTrwGbVhUvRhKJMrLWmyfPcWcIXJomQ8RQ0WjYWqRgVBpwOUc3lDlEZFNRJqiFSWKE4FLKkBu0RIMaMlt1BRXmIfZJY5XiKNJSahwpAbZpdkgyKJ5MSwWzxAT6mLZM0bcaP5tA0f3p178MY2E3av7dNtsNVUGJpYJgVRmknN7LqJKKQuKG9iX5eUM7smMiMGdcWSxgA05iWnusmQRpoSKXhjHA_F2cEtDZJMumah9obMISPM1L3NSs6JpQuOdb_28eZR7IjWXHN4407OAJ4CekDS0krnAoW3ygrFWcIsMcgxsSSTmpTKdbjRldhF307X2VhQmyyJsdRigcI2B_7MmjCoLcmosURRu_xPpKLklin3Q6YsYwl1MRjXcF-3QxXbWLll-S3NyR-SiT3F2zheKVcqKlLCciE1EiEtqZhhzlIieVkIQ5ggiVSH5MKERV1RvneelDQ212gcgVOdY60Lpwii5U1jcbthGHqOTDZToCwr2C2mRFFtmasUpoSJFFfe2v6CC-dU6hQ1poRTYw_k5pTvpUQ0LmWBLlZS1zvd78ntMaM4q7cU4axgjc5Owh9PNxyfm96Mm8aEmkPCe05aMmNlrmnxTSwvQL86Lf02f9uy8U3AQ4tB5cTqvHhlWME4dWuR2KVGs5S8saThq56nasxQEy7lVam80I2f1OzqoFMtFc1dH5lQpa0lwER-aLQ0euTGV93EN_BTb-do-R6aa04MzXCjm0OBuWuIiZw8VdXfbAq1LfXCL4LGouzmu8XruqFLjkRpuaALxvdcPg1mNBWpLIjBZqHXTFbgrRSNg_8pHjpYKRIpjNWUuXEV0m2Wyrdga3YPTlZdswoTK_W-S14EwXAWDeII4sHbcQSqXHCWvFrBUfCCwmgSn8JkGsPk03jcDl4sNif103A6mcezwWgSw4qoK1zDx9now2B2Dr9F53BEYTAfHreDF6PJu-gLrMiCsHQFRwt_HhyfBcFgHEezf3oeTX6NhjHM40E8msej4RxeXgQAAH_6b_dp0Sr3LxitPoTtp-PNOm714eLxsMa3Hp8vt_EaqcWUUNvqQ-sk7J52wm4n7ELY7YdhPwxbW2B33TKRuDenUjhCN9z27feJf6uya-UCa22T_Tp9IG7T3Jp8NHjS6570_H9_tf9vyovvkrKP8PtlHVy-PNutyLVTZPkvRVZNilzvUGT5oMhnuIpkDvl-OotGP09qZHUMs-h9NIsmw2j-KM0j-iRnx_NyrvbKeb1Tzj7L6MvH8WA0gaPpx7gN0eTzMcyjscP-AO9n0w-wht9_iWYRlPAGemdBp9PpBP7iXgdwf3d3f_f1_u4rPG4L24fX3T5cvO5BB173LoO_AwAA__974hQQ +https://cockroachdb.github.io/text/decode.html#eJy0VmFv2zYT_lz9ioO_NHkRt3L8Yggc9IPrqps31y5stWtRFAQtnWQuFMmSlGpnGNAfkV_YXzKQUhJns6wWQ_3BsOjnuTvePXenfh_eojZMihFMZHKlJU02L54DbjFZl4ynqMGisVDVqCDo94FyLj8TpVFRjYQaIpUlilMBG2rAbhBSzGjJLVSUlzgCmWWOl0hjiUmoMOQzsxvSoEgiOTHsGjvoKXWR7HgrbrpanMGrF-89uLHNhD1q-2IfbDUVhiaWSUGUZlIzu2sjCqkLytvYn0rKmd0RmRGDumJJawAa85JT3WZII02JFLw1jtvkHOCWBkkmXbFQe0OmywjTmPh6lIWguqkVCrrmmHZyTa2LrOScWEep-cd4qyh2RGs-cXjmTi4B7i9zi6Sllc5FHQhhheIsYZYY5C7cTGpSKqeOVlfiEH0_Vc7GmtpkQ4ylFgsUtj3wB9aEQW1JRo0litrNN5GKklum3A-Zsowl1MVgnFh83roy1li5Zvk1zckfkokjyWscb5VLFRUpYbmQGomQllTMMGepLrghTJBEqi6pMWFRV5Qf7UUljc01GkfgVOdY68Ipgmj5uTW5gzAMPUcmTQcpywp2jSlRVFvmMoUpYSLFrbd2POHCOZU6RY0p4dTYjru5rvFSIho3skAXa2cDeE9uBhrFWT3hCGcFa3V2Hv7_ouH4u-mmVTUm1HQJ7yFpw4yVuabFd7G8AP3YtfT7_O3LxhcBu4aKyonVefHEsIJx6kYqsRuNZiN5a0rDJ0NP1ZihJlzKq1J5oRvfqdlVp1MtFc1dHZlQpa0lwETe1VoaPbLxVRfxGfw0PNhavobmEyeGZtjopiswt8KYyMl9Vv1WVKhtqdd-ELQm5TDfDV5XDV1yJErLNV0zfmRxtZjRVKSyIAbbhV4zWYHXUrQ2_pt44mClSKQwVlPm2lVIN1kqX4K93v3G1VJhYqU-9oIggmCyjMZxBPH4-SwCVa45S55s4SR4RGE6jy9gvohh_mY2OwserZuT-mmymK_i5Xg6j2FL1BXu4PVy-mq8fA-_Re_hhMJ4NTk9Cx5N5y-id7Ala8LSLZys_XlwehkE41kcLf_peTr_NZrEsIrH8XQVTycrePwhAAD403-7T49WuX856Y0gPLs_bsZxbwQf7g5rfO_u-eM-XiO1mBJqeyPonYeDi3446IcDCAejMByFYW8P7NYtE37Ll8IRBuG-bz9P_BuZ3SkXWG-f7MfpLXGf5sbkncHz4eB86P_76-y_Xnn9Q67sI_xxtw4-Pr48rMidU2T5L0VWbYrcHVBkeavIB7iKZA75crGMpj_Pa2R1CsvoZbSM5pNodSfNE3ovZ8fzcq6Oynl3UM7-ltG717PxdA4ni9fxGUTzt6ewimYO-z94uVy8gh38_ku0jKCEZzC8DPr9fj_wi3sXwNebm683X77efIG7aWFH8HQwgg9Ph9CHp8OPwd8BAAD__1dFKgQ= statement ok SET optimizer_use_histograms = false @@ -100,7 +100,7 @@ SET optimizer_use_histograms = false query T EXPLAIN (OPT, ENV) SELECT * FROM y WHERE u = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvq6e4sA3TQa7leNhCBz0wnXVzZtrF7batSgKgpaOZC4UyZCUYmcY0IfIE-ZJBlJO4myWNWCoLwyL_r7z-51D9XrwEbVhUgxhLJNLLWmyfvMacIPJqmQ8RQ0WjYWqRgVBrweUc3lNlEZFNRJqiFSWKE4FrKkBu0ZIMaMlt1BRXuIQZJY5XiKNJSahwpBrZtdkhyKJ5MSwG2yhp9RFsuWNuMly3oV3bz578M42E_ao7fN9sNVUGJpYJgVRmknN7LaJKKQuKG9iX5WUM7slMiMGdcWSxgA05iWnusmQRpoSKXhjHPfFOcAtDZJMumah9oZMmxFm6t5mJefE0hXHul_HeMsodkRrrji8cicXAI8B3SNpaaVzgcJbZYXiLGGWGOSYWJJJTUrlOtzoShyi76frbKyoTdbEWGqxQGGbA39iTRjUlmTUWKKoXf8nUlFyy5T7IVOWsYS6GIxruK9bW8V2Vm5YfkNz8odk4kjxdo43ypWKipSwXEiNREhLKmaYs5RIXhbCECZIIlWbXJiwqCvKj86TksbmGo0jcKpzrHXhFEG0vG4sbj8MQ8-RyW4KlGUFu8GUKKotc5XClDCR4sZbO15w4ZxKnaLGlHBqbEtuTvleSkTjWhboYiV1vdPjntweM4qzeksRzgrW6Ows_PF8x_G56d24aUyoOSo81_6nrDUzVuaaFqZVBE-JXoR-fVraJvanzH3p-EZg23JQObE6L14YVjBO3Wokdq3RrCVvLGv4YuCpGjPUhEt5WSovduOnNbtsdaqlornrJROqtLUMmMjbxkujR-581Y18BT8NDlbW99FccWJohjvttAXmriImcvJYVX-7KdS21Cu_DBqLcpjvlq_rhi45EqXliq4YP3IBNZjRVKSyIAabxV4zWYE3UjQO_4d47GClSKQwVlPmRlZIt10q34K9-W2drrpmFSZW6mMXvQiC8SIaxRHEo9fTCFS54ix5sYGT4BmFySw-h9k8htmH6bQbPFvtTuqn8Xy2jBejySyGDVGXuIX3i8m70eIz_BZ9hhMKo-X4tBs8m8zeRJ9gQ1aEpRs4Wfnz4PQiCEbTOFr80_Nk9ms0jmEZj-LJMp6Ml_D8SwAA8Kf_dp8OrXL_ktEZQth9PN6t5M4Qvjwc1vjOw_PXfbxGajEl1HaG0DkL--e9sN8L-xD2h2E4DMPOHthduUwk7u2pFI7QD_d9-5Xi36zsVrnAOvtkv1Lvifs0tyofDJ4N-mcD_99f3f-b8uq7pOwj_H5ZB1-fXxxW5NYpsvyXIqsmRW4PKLK8V-QTXEUyh3w7X0STn2c1sjqFRfQ2WkSzcbR8kOYJfZSz43k5V0flvD0oZ59l9On9dDSZwcn8fdyFaPbxFJbR1GF_gLeL-TvYwu-_RIsISngFg4ug1-v1An95bwO4u729u_12d_sNHraFHcLL_hC-vBxAD14OvgZ_BwAA__9A_hRy +https://cockroachdb.github.io/text/decode.html#eJy0VmFv2zYT_lz9ioO_NHkRt3L8Yggc9IPrqps31y5stWtRFAQtnWQuFMmSlGpnGNAfkV-YXzKQUhJns6wCQ_3BsOjnuTvePXenfh_eozZMihFMZHKlJU02r14CbjFZl4ynqMGisVDVqCDo94FyLr8SpVFRjYQaIpUlilMBG2rAbhBSzGjJLVSUlzgCmWWOl0hjiUmoMOQrsxvSoEgiOTHsGjvoKXWR7HgrbrpanMGbVx89uLHNhD1q-2IfbDUVhiaWSUGUZlIzu2sjCqkLytvYX0rKmd0RmRGDumJJawAa85JT3WZII02JFLw1jrvkHOCWBkkmXbFQe0OmywjTmPh6lIWguqkVCrrmmHZyTa2LrOScWEep-cd4qyh2RGu-cHjhTi4BHi5zh6Sllc5FHQhhheIsYZYY5C7cTGpSKqeOVlfiEH0_Vc7GmtpkQ4ylFgsUtj3wR9aEQW1JRo0litrNd5GKklum3A-Zsowl1MVgnFh83roy1li5Zvk1zckfkokjyWscb5VLFRUpYbmQGomQllTMMGepLrghTJBEqi6pMWFRV5Qf7UUljc01GkfgVOdY68Ipgmj5tTW5gzAMPUcmTQcpywp2jSlRVFvmMoUpYSLFrbd2POHCOZU6RY0p4dTYjru5rvFSIho3skAXa2cDeE9uBhrFWT3hCGcFa3V2Hv7_ouH4u-mmVTUm1BwVniv_Y9aGGStzTQvTKYLHRC9CP3ot7RL7Y-a-dHwhsGuwqJxYnRfPDCsYp26sErvRaDaSt6Y1fDb0VI0ZasKlvCqVF7vx3ZpddTrVUtHc1ZIJVdpaBkzkXe2l0SMbX3UhX8BPw4OZ9XU0XzgxNMNGO12BuTXGRE4esuo3o0JtS732w6A1KYf5bvi6auiSI1Faruma8SPLq8WMpiKVBTHYLvaayQq8lqK1-d_FEwcrRSKFsZoy17JCuulS-RLs9e93rpcKEyv1sZcEEQSTZTSOI4jHL2cRqHLNWfJsCyfBEwrTeXwB80UM83ez2VnwZN2c1E-TxXwVL8fTeQxboq5wB2-X0zfj5Uf4LfoIJxTGq8npWfBkOn8VfYAtWROWbuFk7c-D08sgGM_iaPlPz9P5r9EkhlU8jqereDpZwdNPAQDAn_7bfXq0yv0LSm8E4dnDcTOSeyP4dH9Y43v3z5_38RqpxZRQ2xtB7zwcXPTDQT8cQDgYheEoDHt7YLdymfCbvhSOMAj3ffuR4t_K7E65wHr7ZD9S74j7NDcq7w2eDwfnQ__fX2f_9crrH3JlH-GPu3Xw-enlYUXunCLLfymyalPk7oAiyztFPsJVJHPI14tlNP15XiOrU1hGr6NlNJ9Eq3tpntAHOTuel3N1VM67g3L2t4w-vJ2Np3M4WbyNzyCavz-FVTRz2P_B6-XiDezg91-iZQQlvIDhZdDv9_uBX967AG5vbm5vvt3efIP7aWFH8Hwwgk_Ph9CH58PPwd8BAAD__3QxKmY= statement ok SET optimizer_use_multicol_stats = false @@ -108,7 +108,7 @@ SET optimizer_use_multicol_stats = false query T EXPLAIN (OPT, ENV) SELECT * FROM y WHERE u = 3 ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt9u2sgXvq6f4oibJj9Ba8JPq4ioF5S6u-xSqMDtNoqi0WAfm9mMZyYzYweyWqkPkSfMk6xmTBKywbDSqlwgPHzf-fudM-504Ctqw6Tow1AmV1rSZPnhPeAKk0XJeIoaLBoLVY0Kgk4HKOfyhiiNimok1BCpLFGcClhSA3aJkGJGS26horzEPsgsc7xEGktMQoUhN8wuyQZFEsmJYbd4gJ5SF8maN-JG82kbPn049-CNbSbsXtun22CrqTA0sUwKojSTmtl1E1FIXVDexL4uKWd2TWRGDOqKJY0BaMxLTnWTIY00JVLwxjgeirODWxokmXTNQu0NmUNGmKl7m5WcE0sXHOt-7ePNo9gRrbnm8M6dnAE8BfSApKWVzgUKb5UVirOEWWKQY2JJJjUpletwoyuxi76drrOxoDZZEmOpxQKFbQ78mTVhUFuSUWOJonb5r0hFyS1T7odMWcYS6mIwruG-bocqtrFyy_JbmpM_JBN7irdxvFKuVFSkhOVCaiRCWlIxw5ylRPKyEIYwQRKpDsmFCYu6onzvPClpbK7ROAKnOsdaF04RRMubxuJ2wzD0HJlspkBZVrBbTImi2jJXKUwJEymuvLX9BRfOqdQpakwJp8YeyM0p30uJaFzKAl2spK53ut-T22NGcVZvKcJZwRqdnYT_P91wfG56M24aE2r2Cs-1_zlryYyVuaaF2S-Cl0yvQr8_LbUH2C-C3ZaPbwYeWhAqJ1bnxRvDCsapW4_ELjWapeSNpQ3f9DxVY4aacCmvSuUFb_zEZlcHnWqpaO76yYQqbS0FJvJDI6bRIze-6ma-g596O-vje2muOTE0w41-DgXmriMmcvJUVX_DKdS21Au_EBqLspvvFrDrhi45EqXlgi4Y33MJNZjRVKSyIAabBV8zWYG3UjQugC_x0MFKkUhhrKbMja2QbsNUvgVbM3xwwuqaVZhYqfdd9iIIhrNoEEcQD96PI1DlgrPkzQqOglcURpP4FCbTGCZfxuN28GqxOamfhtPJPJ4NRpMYVkRd4Ro-z0afBrNz-C06hyMKg_nwuB28Gk0-RN9gRRaEpSs4Wvjz4PgsCAbjOJr90_No8ms0jGEeD-LRPB4N5_D6IgAA-NN_u0-LVrl_0Wj1IWw_HW_WcqsPF4-HNb71-Hy5jddILaaE2lYfWidh97QTdjthF8JuPwz7YdjaArtrl4nEvUGVwhG64bZvv1b825VdKxdYa5vs1-oDcZvm1uWjwZNe96Tn__ur_V9TXvyQlH2EPy7r4PL12W5Frp0iyxeKrJoUud6hyPJBkc9wFckc8uN0Fo1-ntTI6hhm0cdoFk2G0fxRmkf0Sc6O5-Vc7ZXzeqecfZbRt8_jwWgCR9PPcRuiyddjmEdjh_0ffJxNP8Eafv8lmkVQwjvonQWdTqcT-At8HcD93d393ff7u-_wuC1sH952-3DxtgcdeNu7DP4OAAD__0h8FNQ= +https://cockroachdb.github.io/text/decode.html#eJy0VuFu2zYQ_l09xcF_mgxxK8fDEDjoD9dVN2-uXdhq16IoCFo6yVwokiUp1c4woA-RJ-yTDKSUxF0sq8BQ_zAs-vvujnff3anfh7eoDZNiBBOZXGlJk82L54BbTNYl4ylqsGgsVDUqCPp9oJzLz0RpVFQjoYZIZYniVMCGGrAbhBQzWnILFeUljkBmmeMl0lhiEioM-czshjQokkhODLvGDnpKXSQ73oqbrhZn8OrFew9ubDNhj9q-2AdbTYWhiWVSEKWZ1Mzu2ohC6oLyNvanknJmd0RmxKCuWNIagMa85FS3GdJIUyIFb43jNjkHuKVBkklXLNTekOkywjQmvh5lIahuaoWCrjmmnVxT6yIrOSfWUWr-Md4qih3Rmk8cnrmTS4D7y9wiaWmlc1EHQlihOEuYJQa5CzeTmpTKqaPVlThE30-Vs7GmNtkQY6nFAoVtD_wba8KgtiSjxhJF7ea7SEXJLVPuh0xZxhLqYjBOLD5vXRlrrFyz_Jrm5C_JxJHkNY63yqWKipSwXEiNREhLKmaYs1QX3BAmSCJVl9SYsKgryo_2opLG5hqNI3Cqc6x14RRBtPzcmtxBGIaeI5Omg5RlBbvGlCiqLXOZwpQwkeLWWzuecOGcSp2ixpRwamzH3VzXeCkRjRtZoIu1swG8JzcDjeKsnnCEs4K1OjsPf75oOP5uumlVjQk1R4Xnyv8ta8OMlbmmhTkugodMr0I_ey21HewHwe7LxxcDu4aLyonVefHEsIJx6kYrsRuNZiN5a2rDJ0NP1ZihJlzKq1J5wRvfsdlVp1MtFc1dPZlQpa2lwETe1WIaPbLxVRfzGfwyPJgfX0vziRNDM2z00xWYW2VM5OQ-q347KtS21Gs_EFqTcpjvBrCrhi45EqXlmq4ZP7LAWsxoKlJZEIPtgq-ZrMBrKVoHwJt44mClSKQwVlPm2lZIN2EqX4K9Hv7OFVNhYqU-9qIggmCyjMZxBPH4-SwCVa45S55s4SR4RGE6jy9gvohh_mY2OwserZuT-mmymK_i5Xg6j2FL1BXu4PVy-mq8fA9_RO_hhMJ4NTk9Cx5N5y-id7Ala8LSLZys_XlwehkE41kcLf_reTr_PZrEsIrH8XQVTycrePwhAAD423-7T49WuX9J6Y0gPLs_bsZybwQf7g5rfO_u-eM-XiO1mBJqeyPonYeDi3446IcDCAejMByFYW8P7NYuE37bl8IRBuG-bz9W_JuZ3SkXWG-f7MfqLXGf5sblncHz4eB86P_75-z_Xnn9Q67sI_xxtw4-Pr48rMidU2T5QJFVmyJ3BxRZ3iryG1xFMod8uVhG01_nNbI6hWX0MlpG80m0upPmCb2Xs-N5OVdH5bw7KGd_y-jd69l4OoeTxev4DKL521NYRTOH_QleLhevYAd__hYtIyjhGQwvg36_3w_8At8F8PXm5uvNl683X-BuWtgRPB2M4MPTIfTh6fBj8G8AAAD__9N_Ksg= statement ok RESET reorder_joins_limit @@ -132,7 +132,7 @@ CREATE SEQUENCE seq query T EXPLAIN (OPT, ENV) SELECT * FROM seq ---- -https://cockroachdb.github.io/text/decode.html#eJyUVcFu20gMvfsr5ri7qAMnbptsgx68rhYIEDvd2Am6J2I8oiRuRjNjDuUk_vrFSC6aApHV3ASb75HD90iOx-oeOZJ3n9Tcmwf22lRf_lL4hGbTkM2RlWAUteuiRqPxWGlr_SMExqAZQUfwQSBY7VSlo5IKVY6FbqyonbYNflK-KBLO-CgQjXYRHkkqOESB8RYi7XEAnutUybPtjbta3bxTiy__tsEHbnJylPviZbCwdlEbIe8gMHkmee4DOs-1tn3obaMtyTP4AiLyjkxvAYxlYzX3ETHqHLyzvXV8b84r2CYiFD6JhdwSxSESip22RWMtiN5Y7PQ6hltl6wSUuLXqc_rlUqkfBX2P1I34lAJdy0p1sGRIIKJFI1B4hiYkhXtTudfgL5-bODZaTAVRtGCNTvoL_4nNRWSBQkeBoKX6JVDdWKGQPnxOBRmdaohJ8LZvA50-kOyp3OsS_vPUPztd0qeQ2qRdDlQ6zwjOC-woUqIx3ja1i0AOjA9DViEnyDttj85S8FFKxpgAVnOJnSeSG4D9Y29jTyeTSYvx5jABQaimPeYQNAulLmEO5HJ8atmOv9ulpJ5zZMzB6igDb0uub20EjJWvMdUKXbPz45nSDovBUrehwFJNvcnOJu8vDpj2bXwYNUaj45DpfgZVFMWXrOs3oVrztWtT9NvyvbRNKwIOLYVQgnBZn0Sqyeq0EkEqxlh529vSycm0hTIWyGC9f2hC6_LYTmnxMJiUfdBl0pFcaKSzALlyaBExtpGHXJ2In9XH6as7qdUwbi1EXeDBN0OFpRNEroQfXW2vWkCWhjftEuhtyuv4tHSTGtxYhMB-ozdkjxyeHhrWLvc1ROw3eoekGvfe9Q7-3XqewhpnvIvCmtK4Op82y66V4MXsDk5W17MdGvF87MC70Wh-m83WmVpl_9xly3mmQrOxZE4ibtXiank_u77L1KlazL51n3-enU2n52eT6ceLD-_Pzz9cTM7V1XJ-my2y5VqdqtV6drtWp5ejUfbt6_Xsaql-u_m6fqey5f3vapVdZ_O1-kP9fXuzUBG3l6PxeDweRdw26AyOu6uU_hn9HwAA__-wnkTd +https://cockroachdb.github.io/text/decode.html#eJyUVsFu20gMvfsr5ri7qAMnbptsgx68rhYIEDvd2Am6J4KWKImb0cyYM3ISf_1iRi6aApHV3ISE75FDPj56PFb3JJ6t-aTmNn8Qi3n95S9FT5RvWtYFiQrkg9p1UaPReKxQa_sITsihEKAH6wI4jUbV6FWoSRVUYquD2qFu6ZOyZRlxufUBfI7GwyOHGg5RkFsNnvc0AC8wVvKse-OuVjfv1OLLvyn4wM0mHOW-eBkcBI3HPLA14IStcHjuAxorDeo-9LZFzeEZbAmeZMd5bwFCVatR-oiEsABrdG8d35vzCrb1BKWNwyJJRH6IhIXyNI-2MSiHWZHBjaZiEOs7XZSt1hAipMMfw62ydQQGv9Xqc_zLpVI_HvM9EttgY4quEODGac45gCcdyy2tQOuiOnpTmdfgL1sVOTYY8hp8wEANmdBf-E9sxpMEKNEHcBjqXwI1rQ7s4octuOQcYw0-iiX1baDTB5I9V3us4D_L_XvXJX1ysU1oCuDKWCEwNsCOPUeabtge2EBu3ZDM2ASSHeqje-isD5WQjwCNUlGniagGEPvY29jTyWSSMDY_bI8L3PCeCnAogWOXqAA2BT0ltuPvNjGplYKECtDow8Db4sYkGYFQbRuKtQ6KP2WK_ued5s7dQHPDvcnOJu8vDpj0NjmsqVCOfkh0P4Nq9sFWgs2bUEl8yXIDvi3fS9mkIdCQobgKglTNieeGNUY7hVAL-drq3pZOTqYJKlSSgLb2oXVJ5T5tafkwmFSswyrOkY1rQycBNtWQEQmlyEOuboif1cfpq56UZui3GjyWdNDNUGHxfLGp4EdX00V0JKGVTTKB3qa8jo-mG6chrSZwYje4YX3kaPXQCJrCNuCpX-gdkhvaW9O7-HfreQxrTW6ND4Ic19XY6Cy7NIIXu_uLZ2VHebBy7MeBGY3mt9lsnalV9s9dtpxnyrUbzfmJp61aXC3vZ9d3mTpVi9m37vPPs7Pp9PxsMv148eH9-fmHi8m5ulrOb7NFtlyrU7Vaz27X6vRyNMq-fb2eXS3Vbzdf1-9Utrz_Xa2y62y-Vn-ov29vFsrT9nI0Ho_HI0_blkxO4-4qxf-M_g8AAP__wQVa0Q== # # Test views. @@ -144,7 +144,7 @@ CREATE VIEW v AS SELECT a, b, u, v FROM x, y WHERE b = 3 query T EXPLAIN (OPT, ENV) SELECT * FROM v ---- -https://cockroachdb.github.io/text/decode.html#eJy0Vt9u27YXvo6e4sA3tX-wEzn-YQgcFJjrqpu31C5s9R-KgqCkI5kLRaokpdoZBhR7hlzu6fIkAyk7cVbLXjG0F0VEf9_5-51D9nrwBpVmUgxhLONrJWm8fP4McIVxVDKeoAKD2kBVozyv1wPKufxMCoUFVUioJrIwpOBUwJJqMEuEBFNacgMV5SUOQaap5cVSG6JjKjT5zMySbFAklpxodoNH6Am1kax5I26ymHXh5fP3DryxzYQ5aPtiF2wUFZrGhklBCsWkYmbdRBRS5ZQ3sT-VlDOzJjIlGlXF4sYAFGYlp6rJkEKaECl4Yxzb4uzhlhpJKm2zUDlD-pgRpuvepiXnxNCIY92vQ7xFEFqi0Z84PLUnlwAPAW2RtDTSukDhrLK84CxmhmjkGBuSSkXKwna40ZXYR99N19qIqImXRBtqMEdhmgN_ZE1oVIakVBtSULP8V6S85IYV9g-ZsJTF1MagbcNd3Y5UemPkhmU3NCO_SdY8O7XTVWHLREVCWCakQiKkIRXTzJqJJS9zoQkTJJbFMakwYVBVlB-cpUJqkynUlsCpyrDWhFUDUfJzY2H7vu87jow3E1AYlrMbTEhBlWG2SpgQJhJcOWuH8xbWqVQJKkwIp9ocyc2q3smIKFzKHG2spC52ctiT3WG64KzeUISznDU6O_f_f7HhuNzUZtQUxlQfE91j0pJpIzNF829iOfG5tWnot_nblY1rAh5bCkVGjMryU81yxqldicQsFeql5I0l9U8HjqowRUW4lNdl4VSu3ZSm10edKlnQzPaRiaI0tQSYyI4tIoUOufFVN_Ep_DDYu5NcD_UnTjRNcaObY4HZK4iJjDxU1d1qBSpTqsgtgcai7OfbpWu7oUqOpFAyohHjBy6eBjOKikTmRGOz0Gsmy_FGisbBfx2OLawUsRTaKMrsuAppN0vlWrAzu0cnq65ZhbGR6tAFLzxvPA9GYQDh6NlVAEUZcRafrqDtnVCYTMMLmM5CmL6-uup6J9HmpP4az6aLcD6aTENYkeIa1_BqPnk5mr-HX4P30KYwWow7Xe9kMn0evIMViQhLVtCO3LnXufS80VUYzP_peTL9JRiHsAhH4WQRTsYLePLBAwD43f1v_7VolbnHRWsIfvfheLOOW0P4cH9Y41v33x938QqpwYRQ0xpC69zvX_T8fs_vg98f-v7Q91s7YHvVMhHbV1MpLKHv7_p2-8S9qMy6sIG1dslunW6JuzS7Ju8Nng_65wP32x_d_5py9F1SdhF-v6y9j08u9ytybRVZfqXIqkmR6z2KLLeKfISrSGqRL2bzYPLTtEZWHZgHL4J5MB0Hi3tptumDnC3Pybk6KOf1XjnvZvlmErzdois3dnbQut5JadPzOjBawCK4sjZoF6IulF2o4MV89tLtotNtcN1Hn2t4-3MwDyCCpzC49Lzg3aur0WQK7dmrsAvB9E1na_R_ta3q0uv1ej2PCYGq595F7VhJrTse3N3-dXf75e72C7h3w_qrk9WPm-m2v_xpO3l3e7sB3G8zM4Sz87P-ED6cDaAHZ4OP3g4sZdyg0tA2qsSO93cAAAD__8LDVSA= +https://cockroachdb.github.io/text/decode.html#eJy0Vt1u2zYUvo6e4sA3tQc7keNhCBwUmOuqm7fUKWz1D0VB0NKRzIUiVZJS7AwDij1DLvd0eZKBlJw4q2W3GNqLIqK_7_x-55C9HrxBpZkUQxjL6EpJGi2fPwNcYbQoGI9RgUFtoKxQntfrAeVcXpNcYU4VEqqJzA3JORWwpBrMEiHGhBbcQEl5gUOQSWJ5kdSG6IgKTa6ZWZIaRSLJiWY3eIAeUxvJmjfiJvPLLrx8_t6Ba9tMmL22z7bBRlGhaWSYFCRXTCpm1k1EIVVGeRP7U0E5M2siE6JRlSxqDEBhWnCqmgwppDGRgjfGsSnODm6hkSTSNguVM6QPGWEKI9ePIhNU1b1CQRcc44NcXekiKTgnxlIq_j7ePAgt0ehPHJ7ak3OAh2Q2SFoYaV1UgRCW5ZxFzBCN3IabSEWK3Kqj0ZXYRd8ulbWxoCZaEm2owQyFaQ78kTWhURmSUG1ITs3yq0hZwQ3L7R8yZgmLqI1BW7G4uh2odG3khqU3NCV_SNY8d5XTVW7LREVMWCqkQiKkISXTzJqpmq0JEySS-SGZMWFQlZTvncNcapMq1JbAqUqx0oRVA1HyurGwfd_3HUdG9fTkhmXsBmOSU2WYrRLGhIkYV87a_ryFdSpVjApjwqk2B3KzE-NkRBQuZYY21oPid57s_tM5Z9V2I5xlrNHZqf_jWc1xual6TBVGVB8S3WPSkmkjU0Wzb2I58bmVa-i3-duWjWsCHlooeUqMSrNjzTLGqV2nxCwV6qXkjSX1jweOqjBBRbiUV0XuVK7dlCZXB50qmdPU9pGJvDCVBJhIDy0ihQ5Z-6qa-BR-GuzcSa6H-hMnmiZY6-ZQYPb6YiIlD1V1N2KOyhRq4ZZAY1F28-3Std1QBUeSK7mgC8b3XFoNZhQVscyIxmahV0yW4Y0UjYP_OhxbWCEiKbRRlNlxFdJultK1YGt2v_JaKTEyUu17HAjPG8-CURhAOHp2EUBeLDiLjlfQ9o4oTKbhGUwvQ5i-vrjoekeL-qT6Gl9O5-FsNJmGsCL5Fa7h1WzycjR7D78H76FNYTQfd7re0WT6PHgHK7IgLF5Be-HOvc65540uwmD2X8-T6W_BOIR5OAon83AynsOTDx4AwJ_uf_uvRcvUPUxaQ_C7D8f1Om4N4cP9YYVv3X9_3MYrpAZjQk1rCK1Tv3_W8_s9vw9-f-j7Q99vbYHtVcuEu-ELYQl9f9u32yfuNWbWuQ2stU1263RD3KbZNXlv8HTQPx243_7q_t-UF98lZRfh98va-_jkfLci11aRxReKLJsUud6hyGKjyEe4kiQW-eJyFkx-mVbIsgOz4EUwC6bjYH4vzTZ9kLPlOTmXe-W83inn7SzfTIK3G3Tpxs4OWtc7Kmx6XgdGc5gHF9YG7cKiC0UXSngxu3zpdtHxJrjuo881vP01mAWwgKcwOPe84N2ri9FkCu3LV2EXgumbzsboD5Wt8tzr9Xo9jwmBqufeRe1ISa07Htzd_nN3-_nu9jO4d8P6i5PVz_V021_-tp28u72tAffbzAzh5PSkP4QPJwPowcngo7cFSxg3qDS0jSqw4_0bAAD__2Z2axQ= # # Test tables in user-defined schemas. @@ -157,7 +157,7 @@ CREATE TABLE s.t (a int primary key) query T EXPLAIN (OPT, ENV) SELECT * FROM s.t; ---- -https://cockroachdb.github.io/text/decode.html#eJyUVV1v20YQfA5_xb3FLizDbYrCiJEHRVEBtbJsSExQoygWJ3JJbn28O-8t5Y9fX9xRQRTAFJs3QdiZXe7Mzk0m6gtyIGffq5kr7tnpovn0UeETFtuOTImsBIOoXV-VZZOJ0sa4R_CMXjOCDuC8gDfaqkYHJQ2qEivdGVE7bTp8r1xVRVzhgkAotA3wSNLAvgoKZyDQC47ASx0neTaDdYvNzZm6_nSXivfcZOUo9-VhsbC2QRdCzoJnckzyPAS0jltthtAPnTYkz-AqCMg7KgYHYKw7o3mIiFGX4KwZnOPrcl7BdgGhclEs5EQUxkgo9NpWnTEgemuw1-sYbjPPI1DCg1Ef4j9XSn0b6Gul7sTFFmgTK7XeUEECAQ0WApVj6HxUeLCVfQ1--LmRY6ulaCCIFmzRyvDg37HZgCxQ6SDgtTT_C9R2RsjHH66kigodZwhR8LS3kU3vSV6oftE1_Oto-Hb6pk8-rknbEqi2jhGsE9hRoEhTONO1NgBZKJwfswpZQd5pc_SWvAtSM4YIMJpr7D0R3QDsHgcX-_PFxUXCuGJ_AV6opRcswWsWilvCEsiW-JTYjn-3jU0dl8hYgtFBRr4tuj7ZCBgb12KcFfpll8c7xQwL3lCfUGCopcFmv1z8ernHpG_j_akxFjqMme57UENBXM26_SFUMl-KTdE_1u_QNkkEHAsFX4Nw3Z4HasnoGIkgDWNonBlc6cX5uwRlrJDBOHff-eTykK60uh9tys7rOupI1nfSW4BsPRZEjKly36sX8YP67d2rmZQ0DA8Ggq5w75uxweITRLaGb1tNr5pHlo63KQQGl_I6PoZuVIM7g-DZbfWWzJGHZ4CGtS1dCwGHjd4jqcUXZwcP_3M-i2WdLZwNwpriuVoXk2WXJDi43dHL6ne2w0IcH3vgbZbN1vNpPlf59ONyrsK5qJPsjVaLVX6pVje5Wn1eLs-yN7Ob1SZfTxerXAn4e3xWt-vF9XR9p_6c36kTraab2Wl2epVl02U-Xx_QLVZ_zGe52uTTfLHJF7ONevv3P2-vsmz-1-1yulipk5vb_EzNV19O1Wa-jLU_qd_XN9cRfZVNJpNJlvJKsv8CAAD__3SdR4U= +https://cockroachdb.github.io/text/decode.html#eJyUVt1u20gPva6eYu6afIiDfNvFImjQC9f1At51nMBWiw0WC4KWKGk2o5kJh3J-nn4xIxd1gchq7wyD55BDHh5qMlFfiIN29r2aueKeHRbNp4-KnqjYdtqUxEooiNr1UVk2mSg0xj2CZ_LIBBjAeQFv0KoGg5KGVEkVdkbUDk1H75WrqogrXBAIBdoAj1oa2EdB4QwE_UIj8BJjJc9mMG6xuTlT15_uUvCeW1s5yn15GCyMNmAh2lnwrB1reR4CWsctmiH0Q4dGyzO4CgLxTheDBTDVnUEeImLCEpw1g3V8bc4r2C4QVC4OizgRhTESzVSkeXStRd7PiixuDZWj2NDrouqMAYmQHn8Mt5nnESjhwagP8Z8rpb495mskduJiir4Q0K03utACgUwst3IMnY_qGExlX4MftipybFGKBoKgUEtWhgv_js0GYoEKg4BHaX4I1HZGtI8_XKkrXWCsIUSxpL6NdHpP8qLrF6zhX6eH965P-uRjm9CWoGvrmMA6gZ0OOtL0ww6gLRTOj8lMWyHeoTm6h94FqZlCBBjkmnpNRDUAu8fBxv7_4uIiYVyx3x4vutUvVIJHFh27RCVoW9JTYjv-bhuTOi6JqQSDQUbeFjcmyQiYGtdSrHVU_ClT9L_gje7dDYxu9WCyXy5-vdxj0tt4v6ZMBYYx0X0PanQQVzO2P4VK4kuWK_hz-Q5lk4ZAY4biaxCu2_OgW20w2ilIwxQaZwZbenH-LkGZKmIwzt13Pqk8pC2t7keTsvNYxzlq6zvpJaBtPWZETClyn6sf4gf127tXPSnNMDwYCFjRXjdjhcXzpW0N37qaLqInlo63yQQGm_I6PppunAZ3hsCz2-JWmyNHa4CG0ZauhUDDQu-RuqUXZwcX_3M-i2GdLZwNwqjjuloXnWWXRnCwuz94VnZUiONjHwc2y2br-TSfq3z6cTlX4VzUSfYG1WKVX6rVTa5Wn5fLs-zN7Ga1ydfTxSpXAv6entXtenE9Xd-pP-d36gTVdDM7zU6vsmy6zOfrA7rF6o_5LFebfJovNvlitlFv__7n7VWWzf-6XU4XK3Vyc5ufqfnqy6nazJcx9n_q9_XNdURfZZPJZJIlv5LsvwAAAP__Jj5deQ== # # Test default_transaction_quality_of_service settings. @@ -169,7 +169,7 @@ SET default_transaction_quality_of_service=background query T EXPLAIN (OPT, ENV) VALUES(1); ---- -https://cockroachdb.github.io/text/decode.html#eJyUVcFuG0cMvesr5mgDVmA3RREk8CFNfAiQJgHsGO2JoGa5K9azwzGHK9s65SP8hf6SYmYVxAG02uYmCHyPnPceuculuybNLPG1eyf-RgX9-v2fju7JrwYODakzyuY2Y9VisVw6DEHuICklVALMIMkgBYxujdnZmlxDLQ7B3AbDQK-dtG3BeckG2WPMcMe2hl0VeAmQeUsz8AbLJA9hsu7D5ecT99f7f2rxjpujHeR-9bzYFGNGbywRkrIo28MUMIr2GBaXF1d74bcDBrYHkBYy6YY9uXO3Qn_TqQyxeePcj77fKZW6IaBOTaSEDUgMkyN912kPdsgErRTfSCtRniPhPNrcDiGA4SrQaN0hXBWDs-Xb4M7LP3ufiYNJaUGxsnKfAns2yBTIG7SiMKRi9mSruA_-_LmFY4Xm15ANjXqKNj34T2wxkxq0mA0S2vp_gfohGKfyQxpu2WOZIRfrq24zSu9IttxtsYN_hafXaGx6n4pMGBvgLooSRDHYcOZC4yUMfczAEbykuahwNNINhoNrlSRbp5QLIKB2NGaipAFU7iaFPTs9Pa0Y8btdSMY9b6mBhGpcVKIGODZ0X9kOvzuWpqINKTUQMNvM20rqa4xAaS09lVlhFLs53Kmcs5wCj8cKAvc82ey3099f7TD1bbpbNSWPeS50P4PWnE06xf6XUDV89YIa_lq_57GpJtDcUUgdmHb9i8w9ByzXEWytlNcSJiU9ffGyQpVaUggiN0OqKc91S9ub2aYqCbviI8c02BgBjt3cIVKqlbteo4nn7o-Xe29S9TDfBsjY0i43c4OVrxHHDn6oWj9widQGXdUjMCnKfnw5usUNHQJBUlnhisOBb9AEjWJspIdM00EfkdzTVuLk4n-9elfKhuglZlPksq5RymXZVAue7e7sZo2abcib6KFvfVwsLv7-8vHth0_u6POXqxN38en62F2__fj14tIdnR2_WSyXy-WilueFe3p8fHr89vT4zR2dnRwv_gsAAP__QkEw2g== +https://cockroachdb.github.io/text/decode.html#eJyUVsFu20gMvfsr5pgAceFsF4uiRQ7dNocC3bZA0mD3RNASJXEzGk44lJP41I_IF-ZLihm5aApYVnMLAr5H6vHxjZdLd0WaWMJr906qaxWsuvd_O7qjaj2wr0mdUTK3GasWi-XSofdyC1EpohJgAokG0WNwHSZnHbmaGhy8uQ36gV47aZqMqyQZpApDglu2DnZVUImHxFuagdeYJ7n3k3UfLj6fuH_e_1eKd9wc7CD3q6fFphgSVsYSICqLst1PAYNoj35xcX65F34zoGe7B2kgkW64Infm1lhdtypDqN8497PvD0qldvCoUxMpYQ0S_ORIP3Tagx0SQSN5b6SFKM2RsFJVVjP0AXW3Ngq49lTPYtNokWbwHixDRvwhXBGSk6Ub787yf_ZKhINJbjEOAtxHzxUbJPJ53EYUhpiNMtkq7IM_lSpzrNGqDpKhUU_Bpgf_hS0kUoMGk0FE634L1A_eOOY_pOaGK8wzpGybotuM0juSLbdbbOF_4ekTHJvexSwThhq4DaIEQQw2nDjTjMtOwAEqiXM242CkG_QHTzJKslYpZYBHbWn0RHYDqNxOCnu6Wq0KRqrdHUXjnrdUQ0Q1zipRDRxquitsh7875KaiNSnV4DHZzLfliyk2AqVOesqzzpq_dMpRmKLnMejAc8-Tzf5Y_flqhynfprszVaowzZnuV1DHyaRV7J-FKuYr6Wv4vH5PbVOWQHOBElswbfsXiXv2mJMVrFNKnfhJSVcvXhaoUkMKXuR6iMXlqVxpcz3bVCVim_fIIQ42WoBDOxdESqVy12tc4pn76-XeTCo7TDceEja0883cYPkl49DCT1XL4xhJbdB1CYFJUfbjc-jmbejgCaLKGtfsD7xfEzSKoZYeEk0bfURyT1sJk4f_9fJdLhtCJSGZIudzDZKTZVNW8OR2f_NZ2VBlood-J4TF4vzfLx_ffvjkjj5_uTxx55-ujt3V249fzy_c0enxm8VyuVwuSnlauMeHh8eHb48P39zR6cnx4nsAAAD__x0_Rs4= statement ok SET default_transaction_quality_of_service=critical @@ -177,7 +177,7 @@ SET default_transaction_quality_of_service=critical query T EXPLAIN (OPT, ENV) VALUES(1); ---- -https://cockroachdb.github.io/text/decode.html#eJyUVcFuG0cMvesr5mgDVmA3RREk8CFNfAiQJgHsGO2JoGa5K9azwzGHK9s65SP8hf6SYmYVxAG02uYmCHyPnPceuculuybNLPG1eyf-RgX9-v2fju7JrwYODakzyuY2Y9VisVw6DEHuICklVALMIMkgBYxujdnZmlxDLQ7B3AbDQK-dtG3BeckG2WPMcMe2hl0VeAmQeUsz8AbLJA9hsu7D5ecT99f7f2rxjpujHeR-9bzYFGNGbywRkrIo28MUMIr2GBaXF1d74bcDBrYHkBYy6YY9uXPnlY09hjfO_ej6nVCpGwLq1DxK2IDEMDnQd5X2YIdM0EpxjbQS5TkSzqPJ7RACGK4CjcYdwlUpOFu-De68_LP3mTiYlBYUKyv3KbBng0yBvEErCkMqVk-2ivvgz59bOFZofg3Z0KinaNOD_8QWM6lBi9kgoa3_F6gfgnEqP6Thlj2WGXIxvuo2o_SOZMvdFjv4V3h6icam96nIhLEB7qIoQRSDDWcuNF7C0McMHMFLmosKRyPdYDi4VEmydUq5AAJqR2MmShpA5W5S2LPT09OKEb_bhGTc85YaSKjGRSVqgGND95Xt8LtjaSrakFIDAbPNvK2kvsYIlNbSU5kVRrGbw53KMcsp8HiqIHDPk81-O_391Q5T36a7VVPymOdC9zNozdmkU-x_CVXDV--n4a_1ex6bagLNHYXUgWnXv8jcc8ByG8HWSnktYVLS0xcvK1SpJYUgcjOkmvJct7S9mW2qkrArPnJMg40R4NjNHSKlWrnrNZp47v54ufcmVQ_zbYCMLe1yMzdY-RZx7OCHqvXzlkht0FU9ApOi7MeXo1vc0CEQJJUVrjgc-AJN0CjGRnrINB30Eck9bSVOLv7Xq3elbIheYjZFLusapVyWTbXg2e7Obtao2Ya8iR760sfF4uLvLx_ffvjkjj5_uTpxF5-uj931249fLy7d0dnxm8VyuVwuanleuKfHx6fHb0-P39zR2cnx4r8AAAD__xVIMAU= +https://cockroachdb.github.io/text/decode.html#eJyUVsFu20gMvfsr5pgAceFsF4uiRQ7dNocC3bZA0mD3RNASJXEzGk44lJP41I_IF-ZLihm5aApYVnMLAr5H6vHxjZdLd0WaWMJr906qaxWsuvd_O7qjaj2wr0mdUTK3GasWi-XSofdyC1EpohJgAokG0WNwHSZnHbmaGhy8uQ36gV47aZqMqyQZpApDglu2DnZVUImHxFuagdeYJ7n3k3UfLj6fuH_e_1eKd9wc7CD3q6fFphgSVsYSICqLst1PAYNoj35xcX65F34zoGe7B2kgkW64InfmKmXjCv0b5352_UGo1A4edWoeJaxBgp8c6IdKe7BDImgkb420EKU5ElaqymKGPqDulkYB157qWWwaDdIM3oNlyIg_hCsycrJ0491Z_s9eiXAwyS3GQYD76Llig0Q-j9uIwhCzTSZbhX3wp1JljjVa1UEyNOop2PTgv7CFRGrQYDKIaN1vgfrBG8f8h9TccIV5hpRNU3SbUXpHsuV2iy38Lzx9gGPTu5hlwlADt0GUIIjBhhNnmnHZCThAJXHOZhyMdIP-4EFGSdYqpQzwqC2NnshuAJXbSWFPV6tVwUi1u6Jo3POWaoioxlklqoFDTXeF7fB3h9xUtCalGjwmm_m2fDHFRqDUSU951lnzl045CFP0PMYceO55stkfqz9f7TDl23R3pkoVpjnT_QrqOJm0iv2zUMV8JXsNn9fvqW3KEmguUGILpm3_InHPHnOugnVKqRM_KenqxcsCVWpIwYtcD7G4PJUrba5nm6pEbPMeOcTBRgtwaOeCSKlU7nqNSzxzf73cm0llh-nGQ8KGdr6ZGyy_Yxxa-KlqeRojqQ26LiEwKcp-fA7dvA0dPEFUWeOa_YHXa4JGMdTSQ6Jpo49I7mkrYfLwv16-y2VDqCQkU-R8rkFysmzKCp7c7m8-KxuqTPTQr4SwWJz_--Xj2w-f3NHnL5cn7vzT1bG7evvx6_mFOzo9frNYLpfLRSlPC_f48PD48O3x4Zs7Oj05XnwPAAD__75LRfk= # # Test recursive table references from foreign keys. @@ -194,7 +194,7 @@ CREATE TABLE z ( query T EXPLAIN (OPT, ENV) SELECT * FROM z; ---- -https://cockroachdb.github.io/text/decode.html#eJy0VtFv27YTfq7-ioNf6vxgF3LywxA46IPrqoM31w5stWhQFAQtnWTOFMmQlGJ72P8-kHISZ5XsFUPzEETMfXf33X13ZL8Pn1EbJsUQxjLZaEmT9ft3gFtMViXjKWqwaCxUtVUQ9PtAOZcPRGlUVCOhhkhlieJUwJoasGuEFDNacgsV5SUOQWaZwyXSWGISKgx5YHZNDlYkkZwYtscz8JS6THa81W6ynPfg4_s7b3zwzYQ96fv62NhqKgxNLJOCKM2kZnbXBhRSF5QHyyhuhN-XlDO7IzIjBnXFEoS3kGhmWUL5DcBz1EeHGvOSU92Wj0aaEil4a0KPVWrAlgZJJl3XUHtH5pwTZuomZyXnxNIVx7pxp3C-FMxYc8_hrTtppElLK10IFN4rKxRnCbPEIMfEkkxqUirX6tZQogl-TNf5WFGbrImx1GKBwrYn_sKbMKgtyaixRFG7_legouSWKfeHTFnGEupyMK7xvm5nKn1wsmf5nubkD8nah6gOulWuTFSkhOVCaiRCWlIxw5ybRPKyEIYwQRKpzkmFCYu6ovzkUClpbK7ROACnOsdaE04NRMuH1sIOwjD0GJkcJkFZVrA9pkRRbZmrEqaEiRS33ttp3sIFlTpFjSnh1Ngz3JzqvYyIxrUs0OVK6mKnpyO5ZWYUZ_WqIpwVrDXYZfj_6wPGc9OHUdOYUHNOdC9Ba2aszDUtfgjlxef3p6U_Fu9YNr4JeG4pqJxYnRdvDCsYp243ErvWaNaSt5Y0fHPloRoz1IRLuSmVV7nxU5ptzgbVUtHc9ZEJVdpaAkzk5xaRRm95iFU38S38ctW4k3wPzT0nhmZ40M25xNxdxEROnqvqrzeF2pZ65ZdAa1Ga8W7pum7okiNRWq7oivETN1CLG01FKgtisF3oNZIVuJeidfA_xWNnVopECmM1ZW5chXSbpfItOJrds5NV16zCxEp96qYXQTBeRKM4gnj0bhqBKlecJW-20A1eUZjM4muYzWOYfZpOe8Gr1eGk_hrPZ8t4MZrMYtgStcEd3C4mH0eLO_g9uoMuhdFyfNELXk1m76MvsCUrwtItdFf-PLi4CYLRNI4W_4w8mf0WjWNYxqN4sown4yW8_hoAAPzpf7ufDq1y_8roDCHsPR8f1nFnCF-fDmv7ztP3t2N7jdRiSqjtDKFzGQ6u--GgHw4gHAzDcBiGnSNjd9UykbjnUykcYBAex_b7xD-t7E65xDrHYL9OH4HHMLcmnxxeXg0ur_z__ur9V8qrn0LZZ_jzWAffXt80K3LnFFl-p8iqTZG7BkWWj4p8YVeRzFl-mC-iya-z2rK6gEX0IVpEs3G0fJJmlz7L2eG8nKuTct41yrmV5d6xVJvvaGrM2ojuG4iqTQPTbPOSo8asieWuW7ay2bezib7cTkeTGXTnt3EPotnnC1hGU2f7P_iwmH-E_U3Q7_f7gX967IO_AwAA__8CuT4y +https://cockroachdb.github.io/text/decode.html#eJy0VlFv2zYQfq5-BeGXJoNdyMkwBA764Lrq4M21C1stWhQFQUsnmTNFskdKsT3svw-knMRZJavF0DwEEXPf3X133x05GJAPgIYrOSITlWxRsWTz-hWBHSTrkosUkFgwllS1VRAMBoQJoe6oRtAMgTJDlbZUCybJhhliN0BSyFgpLKmYKGFEVJY5XKKMpSZh0tA7bjf0aEUTJajhB-iAp8xlshetdtPVok_evv7kjY--ubRnfd-cGltk0rDEciWpRq6Q230bUCosmAhWUdwI_1oywe2eqowawIonQF6SBLnlCRO3hDxGvXeIkJeCYVs-CCylSorWhO6r1IAtDdBMua4BekemywlHSHxjykIyPDYNJFsLSDuxphZIVgpBrYPU-HM4X0ZurPkqyEt30lgiVlrlQtSJUF5owRNuqQHh0s0U0lI7mbSGkk3w01I5H2tmkw01llkoQNr2xJ94kwbQ0owZSzWzm-8CFaWwXLs_VMoznjCXg3Gi8XXrqPTRyYHnB5bTvxRvH8A66E67MjGZUp5LhUClsrTihjs3dbMN5ZImSnfJjEsLWDFxdiC1MjZHMA4gGOZQa8KpgaK6ay3sMAxDj1HJcYq05QU_QEo1Q8tdlSClXKaw897O85YuqMIUEFIqmLEd3NzEeBlRhI0qwOXaKX4fyS1CowWv1xwVvOCtwa7CX2-OGM8Nj2OKkDDTJbqnoA03VuXIih9CefH53WvZj8U7lY1vAnQtFJ1Ti3nxwvCCC-b2KrUbBLNRorWk4YtrD0XIAKlQaltqr3LjpzTbdgZFpVnu-silLm0tAS7zrkWE4C2PseomviS_XTfuJN9D81VQwzI46qYrMXePcZnTx6r6q1ED2hLXfgm0FqUZ75au6waWAqhGtWZrLs7cXi1ukMlUFdRAu9BrJC_goGTr4L-PJ86slImSxiLjblylcpul8i04md3vvFYqSKzCc68EGQSTZTSOIxKPX80iosu14MmLHbkInjEyncc3ZL6Iyfz9bNYPnq2PJ_XXZDFfxcvxdB6THdVb2JN3y-nb8fIT-TP6RC4YGa8ml_3g2XT-OvpIdnRNebojF2t_HlzeBsF4FkfL_0aezv-IJjFZxeN4uoqnkxV5_jkghJC__W_302NV7l8ovREJ-4_Hx3XcG5HPD4e1fe_h-8upPQKzkFJmeyPSuwqHN4NwOAiHJByOwnAUhr0TY3fVculv-FI6wDA8je33iX-W2b12ifVOwX6d3gNPYW5NPji8uh5eXfv__dP_v5TXP4Wyz_DnsQ6-PL9tVuTeKbL8RpFVmyL3DYos7xX5xK6imbN8s1hG09_ntWV1SZbRm2gZzSfR6kGaF-xRzg7n5VydlfO-Uc6tLA-Opd5-QxMhayN6aCCqtw1Ms-1TjghZE8v9RdnK5tDOJvr4bjaezsnF4l3cJ9H8wyVZRTNn-wt5s1y8JYfbYDAYDAL_9DgE_wYAAP__kGtUJg== # A foreign key cycle shouldn't cause infinite recursion. statement ok @@ -203,7 +203,7 @@ ALTER TABLE y ADD CONSTRAINT fk FOREIGN KEY (v) REFERENCES z (pk); query T EXPLAIN (OPT, ENV) SELECT * FROM y; ---- -https://cockroachdb.github.io/text/decode.html#eJy0VtFv27YTfq7-ioNf6vwQF3LywxA46IPrqoM31y5stWhRFAQtnWTOFMmSlGp52P8-kHIad5WsFUPzEETMfXf33X135GgE71AbJsUEZjLZa0mT3csXgAdMtiXjKWqwaCxUjVUQjEZAOZdfiNKoqEZCDZHKEsWpgB01YHcIKWa05BYqykucgMwyh0ukscQkVBjyhdkdOVmRRHJi2BF74Cl1mdS8026-WV3D65cfvPHJNxP2ou-7c2OrqTA0sUwKojSTmtm6CyikLigPNlHcCv9cUs5sTWRGDOqKJQjPIdHMsoTye4DHqA8ONeYlp7orH400JVLwzoQeqtSCLQ2STLquofaOTJ8TZpomZyXnxNItx6Zxl3C-FMxY85nDc3fSSpOWVroQKLxXVijOEmaJQY6JJZnUpFSu1Z2hRBv8nK7zsaU22RFjqcUChe1O_BtvwqC2JKPGEkXt7l-BipJbptwfMmUZS6jLwbjG-7r1VPrk5MjyI83JH5J1D1ET9KBcmahICcuF1EiEtKRihjk3ieRlIQxhgiRS9UmFCYu6ovziUClpbK7ROACnOsdGE04NRMsvnYUdh2HoMTI5TYKyrGBHTImi2jJXJUwJEykevLfLvIULKnWKGlPCqbE93JzqvYyIxp0s0OVKmmKnlyO5ZWYUZ82qIpwVrDPYTfj_uxPGc9OnUdOYUNMnum9BO2aszDUtfgjlxef3p6U_Fu9cNr4J2LcUVE6szotnhhWMU7cbid1pNDvJO0saPrv1UI0ZasKl3JfKq9z4Kc32vUG1VDR3fWRClbaRABN53yLS6C1PsZomPodfblt3ku-h-cyJoRmedNOXmLuLmMjJY1X99aZQ21Jv_RLoLEo73i1d1w1dciRKyy3dMn7hBupwo6lIZUEMdgu9QbICj1J0Dv7beObMSpFIYaymzI2rkG6zVL4FZ7PbO1lNzSpMrNSXbnoRBLN1NI0jiKcvFhGocstZ8uwAw-AJhfkyvoPlKobl28XiOniyPZ00X7PVchOvp_NlDAei9ljDm_X89XT9AX6PPsCQwnQzu7oOnsyXL6P3cCBbwtIDDLf-PLi6D4LpIo7W_4w8X_4WzWLYxNN4vonnsw08_RgAAPzpf7ufAa1y_8oYTCC8fjw-rePBBD5-PWzsB1-_P53ba6QWU0LtYAKDm3B8NwrHo3AM4XgShpMwHJwZu6uWicQ9n0rhAOPwPLbfJ_5pZWvlEhucg_06fQCew9ya_Orw5nZ8c-v_99f1f6W8_SmUfYY_j3Xw6el9uyKPTpFq_50kNWZdojy2iFLtH1R5Zpjt4dVqHc1_XTZGGrMrWEevonW0nEWbhxTqYdkp22OrbDvZ1I5N-R2ZqotK3UKlbGFSk4pkzvIbPlUbm8OQ9pShFXYcqv3jULt4fqiri0Ndd1cnev9mMZ0vYbh6E19DtHx3BZto4Wz_B6_Wq9dQ3wej0WgU-CdLHfwdAAD__7P6TfQ= +https://cockroachdb.github.io/text/decode.html#eJy0Vl1v2zYUfa5-xYVf6gxxISfDEDjog-uqg7fULmy1aFEUBC1dyZwpkiUp1fKw_z6QchpnlawWQ_MQRMw99_PcQ45G8A61YVJMYCaTnZY02b58AbjHZFMynqIGi8ZC1VgFwWgElHP5hSiNimok1BCpLFGcCthSA3aLkGJGS26horzECcgsc7hEGktMQoUhX5jdkqMVSSQnhh2wB55Sl0nNO-3m6-UlvH75wRsffTNhz_q-OTW2mgpDE8ukIEozqZmtu4BC6oLyYB3FrfDPJeXM1kRmxKCuWILwHBLNLEsovwV4iHrvUGNecqq78tFIUyIF70zovkst2NIgyaSbGmrvyPQ5YRoTP5iyEFQfh4aCbjimvVjTECQrOSfWQRr8OZxvIzPWfObw3J20toiWVroQTSKEFYqzhFlikLt0M6lJqRxNOkOJNvhpq5yPDbXJlhhLLRYobHfij7wJg9qSjBpLFLXb7wIVJbdMuT9kyjKWUJeDcaTxfevp9NHJgeUHmpO_JOtewCboXrk2UZESlgupkQhpScUMc26aYRvCBEmk6qMZExZ1RfnZhVTS2FyjcQBOdY4NJxwbiJZfOhs7DsPQY2Ry3CJlWcEOmBJFtWWuS5gSJlLce2_n6xYuqNQpakwJp8b21OY2xtOIaNzKAl2uveT3kZwQGsVZI3OEs4J1BrsKf705Ynxt-rimGhNq-kj3GLRlxspc0-KHUJ58Xnst_bF4p7TxQ8A-QVE5sTovnhlWME6drhK71Wi2kne2NHx27aEaM9SES7krlWe58Vua7XqDaqlo7ubIhCptQwEm8j4h0ugtj7GaIT6H365bNcnP0HzmxNAMj7zpS8zdY0zk5KGr_mpUqG2pN14EOpvSjnei66ahS45EabmhG8bP3F4dbjQVqSyIwW6iN0hW4EGKzsV_G8-cWSkSKYzVlLl1FdIpS-VHcLK733mtVJhYqc-9EkQQzFbRNI4gnr64i0CVG86SZ3sYBk8ozBfxDSyWMSze3t1dBk82x5Pma7ZcrOPVdL6IYU_UDmt4s5q_nq4-wJ_RBxhSmK5nF5fBk_niZfQe9mRDWLqH4cafBxe3QTC9i6PVfyPPF39EsxjW8TSer-P5bA1PPwYAAH_73-5nQKvcv1AGEwgvH46PcjyYwMevh4394Ov3p1N7jdRiSqgdTGBwFY5vRuF4FI4hHE_CcBKGgxNjd9Uy4W_4UjjAODyN7fXEP8tsrVxig1Owl9N74CnMyeRXh1fX46tr_79_Lv9vyZufUrLP8OdVHXx6etvOyINjpNp9Q0mNWRcpDy2kVLt7Vp4YZjt4tVxF898XjZHG7AJW0atoFS1m0fo-hXpYdtL20ErbzmpqV035TTFVVyl1SyllSyU1qUjmLB_VU7VVsx_Snja0wg5DtXtYahfPL3V1dqnr7u5E79_cTecLGC7fxJcQLd5dwDq6c7a_wKvV8jXUt8FoNBoF_slSB_8GAAD__w9mY-g= # Check that we remove histograms from statistics correctly. @@ -259,4 +259,4 @@ ALTER TABLE b INJECT STATISTICS '[ query T EXPLAIN (OPT, ENV) SELECT * FROM b ---- -https://cockroachdb.github.io/text/decode.html#eJy8Vk9vG7kPPXc-BeFL0x_iwHH6200d9OA6LuBd1y7iadGiKAjNDD3mRiMpksb5s9jvvpDGaZ2Fx24PuzkEicRHUo_k43S78JGsY60GMNL5tdUiX12-AbqjPKtZFmTBk_OwbqySpNsFIaW-RWPJCEsoHGrj0UihYCUc-BVBQUtRSw9rIWsagF4uAy7XzqPLhXJ4y36FGyvMtUTHD3QAXoiQyb1stZss5sfw7vJzNN74ZuX3-j7fNvZWKCdyz1qhsawt-_s2oNK2EjJZjNOd8JtaSPb3qJfoyK45J3gNuWXPuZAXAN-jPjq0VNZS2LZ8LIkCtZKtCT2ytANbO8KlDlUjGx25Q07YNUVe1lKiF5mkpnD7cJEKdt7dSHgdTnY-U9RehxCkoleujOScPTqSlHtcaou1CaVuDaV2wbefG3xkwucrdF54qkj59sSfeFOOrMelcB6N8KsfAlW19GzCH7rgJeci5OBC4SNvB5jeOHng8kGU-Ifm9iFqgt6ZQJNQBXKptCVU2uOaHQc3uZZ1pRyywlybQ63CypNdC7l3qIx2vrTkAkAKW1LTE6Eb0OrbVmJPe71exOh8MwnGc8UPVKAR1nNgiQpkVdBd9Lb_3SoE1bYgSwVK4fyBt4Wuj22Ella6opArNmQX-yMFMXNGciNVKLni1mD93svzDSa-zW5GzVIu3KGmewpasfO6tKL6KVRsvqifXvxcvO22iUWgQ6JgSvS2rE4cVyxF0Eb0K0tupWUrpb2Tswi1tCSLUuvr2sQud3FKl9cHg1ptRBnqyMrUvmkBVuUhIbIULTexmiK-hl_OdmpSrKG7kejEkjZ9cyixsItYlfid1bjeDFlf2yyKQCspu_FBdEM1bC0JjdWZyFju2UAtbqxQha7QUXujN0iu6EGr1sH_kI6CWa1yrZy3gsO4Kh2UZR1LsDW7Byer4WxNudd236ZXSTK6Gg_TMaTDN9MxmDqTnJ9kcJQ8y-DNfD6F2TyF2Yfp9Dh5ZvUtFzCZpefx9ONkMQmgRwu4HL8dfpimUCu-qaMEcHH04jh5NprPFunVcDJLIUNzTffw_mrybnj1GX4ff4ajxu9wMQq2k9nl-BNkmCEXd3CUxfPkxUWSDKfp-OqfiU5mv41HKSzSYTpZpJPRAp5_SQAA_oy_w09HrMv4UdIZwOnx9-ONencG8OXbYbzIOt_-_7ptb0l4KlD4zgA6_V6_3z3td3t9OD0fnL0c9F-d_P_Xl6_O-p0tTFjQrPLw0VWrgOtvXUYNip9j_t6E7DrbUCWqeIYYdjjik7sgz48ue1sXQXYfz09Pe71489fxHkZ6P8JIrNC_yErvv2NlQ0ny9flFkow_vZ8OJzM4mr9Pj2E8-_gCFuNpaKj_wdur-TvILpJut9tN4sbMkr8DAAD__1Fj5XE= +https://cockroachdb.github.io/text/decode.html#eJy8Vk9vG7kPPXc-BeFL0x_iwHH6200d9OA6LuBd1y7iadGiKAjNDD3mRiMpksb5s9jvvpDstO7C40kPuzkEicRHch4fSXW78JGsY60GMNL5tdUiX12-AbqjPKtZFmTBk_Ow3lglSbcLQkp9i8aSEZZQONTGo5FCwUo48CuCgpailh7WQtY0AL1cBlyunUeXC-Xwlv0Kt1aYa4mOH6gFXoiQyb1stJss5sfw7vJzNN76ZuUP-j7fNfZWKCdyz1qhsawt-_smoNK2EjJZjNO98JtaSPb3qJfoyK45J3gNuWXPuZAXAN-jPjq0VNZS2KZ8LIkCtZKNCT2ytAdbO8KlDlUjGx25NidsKY-FqSsl7LZopEQmqWjFuo1AlrWU6ANkgz-EizSy8-5GwutwspciUXsdQmwSQa6M5Jw9OpIh3aW2WJsgk8ZQah98l6rgIxM-X6HzwlNFyjcn_oM35ch6XArn0Qi_ehKoqqVnE_7QBS85FyEHF0QTeWtheuvkgcsHUeIfmpsbcBP0zgSahCqQS6UtodIe1-w4uNkU2yErzLVpkxkrT3Yt5MGGNNr50pILAClsSRtNBDWg1beNxJ72er2I0fm2i4znih-oQCOs58ASFciqoLvo7fB3qxBU24IsFSiF8y3fFjomyggtrXRFIddW8cdIYRA6I3kz5lByxY3B-r2X51tM_Da7bVNLuXBtovsRtGLndWlF9VOoKL44e734uXi7solFoLaBYkr0tqxOHFcsRZir6FeW3ErLRkp7J2cRamlJFqXW17WJKnexS5fXrUGtNqIMdWRlar-RAKuybRBZipbbWJsivoZfzvbOpFhDdyPRiSVtddOWWNhjrEr8zmpcjYasr20Wh0AjKfvxYeiGathaEhqrM5GxPLC9GtxYoQpdoaNmoW-QXNGDVo2N_yEdBbNa5Vo5bwWHdlU6TJZ1LMFO7z5xrawp99oeeiWoJBldjYfpGNLhm-kYTJ1Jzk8yOEqeZfBmPp_CbJ7C7MN0epw8s_qWC5jM0vN4-nGymATQowVcjt8OP0xTqBXf1HEEcHH04jh5NprPFunVcDJLIUNzTffw_mrybnj1GX4ff4ajjd_hYhRsJ7PL8SfIMEMu7uAoi-fJi4skGU7T8dU_E53MfhuPUlikw3SySCejBTz_kgAA_Bl_h5-OWJfxQdMZwOnx9-Pt9O4M4Mu3w3iRdb79_3XX3pLwVKDwnQF0-r1-v3va7_b6cHo-OHs56L86-f-vL1-d9Ts7mLCgWcV3Qa0Crr9zGWdQfMr5exOy6-xClajiGWLY4Yg_3IXx_Oiyt3MRxu7j-elprxdv_jo-wEjvKYzECv2LrPT-O1a2lCRfn18kyfjT--lwMoOj-fv0GMazjy9gMZ4GQf0P3l7N30F2kXS73W4SN2aW_B0AAP__pt37ZQ== diff --git a/pkg/sql/opt/exec/execbuilder/tests/5node/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/5node/generated_test.go index 23293121772b..ea46feba978c 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/5node/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/5node/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/fakedist-disk/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/fakedist-disk/generated_test.go index 73460bd85082..909c70b4345c 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/fakedist-disk/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/fakedist-vec-off/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/fakedist-vec-off/generated_test.go index b25001a7d460..a67403b1090c 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/fakedist-vec-off/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/fakedist/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/fakedist/generated_test.go index bd0ffe728494..cf90fc6e2417 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/fakedist/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/fakedist/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local-legacy-schema-changer/generated_test.go index bbee57a0c549..36d653cbc5e2 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local-legacy-schema-changer/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/local-vec-off/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local-vec-off/generated_test.go index b249febe8c57..f9d490fe22fc 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local-vec-off/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local-vec-off/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go index a2832beb63b7..d1c90e54a4ae 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go @@ -60,6 +60,9 @@ func runExecBuildLogicTest(t *testing.T, file string) { serverArgs := logictest.TestServerArgs{ DisableWorkmemRandomization: true, ForceProductionValues: true, + // Disable the direct scans in order to keep the output of EXPLAIN (VEC) + // deterministic. + DisableDirectColumnarScans: true, } logictest.RunLogicTest(t, serverArgs, configIdx, filepath.Join(execBuildLogicTestDir, file)) } diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index dfc6d8a5afb7..92a88921e3c3 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -70,6 +70,7 @@ go_library( "//pkg/util", "//pkg/util/admission", "//pkg/util/admission/admissionpb", + "//pkg/util/buildutil", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/intsets", diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index e6b95eee4b88..8c224ed7a1b6 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -75,10 +75,11 @@ type KVBatchFetcherResponse struct { // KVs, if set, is a slice of roachpb.KeyValue, the deserialized kv pairs // that were fetched. KVs []roachpb.KeyValue - // BatchResponse, if set, is a packed byte slice containing the keys and - // values. An empty BatchResponse indicates that nothing was fetched for the - // corresponding ScanRequest, and the caller is expected to skip over the - // response. + // BatchResponse, if set, is either a packed byte slice containing the keys + // and values (for BATCH_RESPONSE scan format) or serialized representation + // of a coldata.Batch (for COL_BATCH_RESPONSE scan format). An empty + // BatchResponse indicates that nothing was fetched for the corresponding + // ScanRequest, and the caller is expected to skip over the response. BatchResponse []byte // spanID is the ID associated with the span that generated this response. spanID int @@ -646,37 +647,14 @@ func (rf *Fetcher) startScan(ctx context.Context) error { return err } -// setNextKV sets the next KV to process to the input KV. needsCopy, if true, -// causes the input kv to be deep copied. needsCopy should be set to true if -// the input KV is pointing to the last KV of a batch, so that the batch can -// be garbage collected before fetching the next one. -// gcassert:inline -func (rf *Fetcher) setNextKV(kv roachpb.KeyValue, needsCopy bool) { - if !needsCopy { - rf.kv = kv - return - } - - // If we've made it to the very last key in the batch, copy out the key - // so that the GC can reclaim the large backing slice before we call - // NextKV() again. - kvCopy := roachpb.KeyValue{} - kvCopy.Key = make(roachpb.Key, len(kv.Key)) - copy(kvCopy.Key, kv.Key) - kvCopy.Value.RawBytes = make([]byte, len(kv.Value.RawBytes)) - copy(kvCopy.Value.RawBytes, kv.Value.RawBytes) - kvCopy.Value.Timestamp = kv.Value.Timestamp - rf.kv = kvCopy -} - // nextKey retrieves the next key/value and sets kv/kvEnd. Returns whether the // key indicates a new row (as opposed to another family for the current row). func (rf *Fetcher) nextKey(ctx context.Context) (newRow bool, spanID int, _ error) { - ok, kv, spanID, needsCopy, err := rf.kvFetcher.nextKV(ctx, rf.mvccDecodeStrategy) + ok, kv, spanID, err := rf.kvFetcher.nextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return false, 0, ConvertFetchError(&rf.table.spec, err) } - rf.setNextKV(kv, needsCopy) + rf.kv = kv if !ok { // No more keys in the scan. diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index 5b76766799c9..b7796f65c508 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -21,10 +21,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/errors" @@ -136,6 +138,12 @@ type txnKVFetcher struct { // this scans inside of DistSender. batchBytesLimit rowinfra.BytesLimit + // scanFormat indicates the scan format that should be used for Scans and + // ReverseScans. With COL_BATCH_RESPONSE scan format, indexFetchSpec must be + // set. + scanFormat roachpb.ScanFormat + indexFetchSpec *fetchpb.IndexFetchSpec + reverse bool // lockStrength represents the locking mode to use when fetching KVs. lockStrength lock.Strength @@ -261,9 +269,11 @@ type newTxnKVFetcherArgs struct { // is non-nil. func newTxnKVFetcherInternal(args newTxnKVFetcherArgs) *txnKVFetcher { f := &txnKVFetcher{ - sendFn: args.sendFn, + sendFn: args.sendFn, + // Default to BATCH_RESPONSE. The caller will override if needed. + scanFormat: roachpb.BATCH_RESPONSE, reverse: args.reverse, - lockStrength: getKeyLockingStrength(args.lockStrength), + lockStrength: GetKeyLockingStrength(args.lockStrength), lockWaitPolicy: getWaitPolicy(args.lockWaitPolicy), lockTimeout: args.lockTimeout, acc: args.acc, @@ -404,8 +414,21 @@ func (f *txnKVFetcher) fetch(ctx context.Context) error { ba.Header.LockTimeout = f.lockTimeout ba.Header.TargetBytes = int64(f.batchBytesLimit) ba.Header.MaxSpanRequestKeys = int64(f.getBatchKeyLimit()) + if buildutil.CrdbTestBuild { + if f.scanFormat == roachpb.COL_BATCH_RESPONSE && f.indexFetchSpec == nil { + return errors.AssertionFailedf("IndexFetchSpec not provided with COL_BATCH_RESPONSE scan format") + } + } + if f.indexFetchSpec != nil { + ba.IndexFetchSpec = f.indexFetchSpec + // SQL operators assume that rows are always complete in + // coldata.Batch'es, so we must use the WholeRowsOfSize option in order + // to tell the KV layer to never split SQL rows across the + // BatchResponses. + ba.Header.WholeRowsOfSize = int32(f.indexFetchSpec.MaxKeysPerRow) + } ba.AdmissionHeader = f.requestAdmissionHeader - ba.Requests = spansToRequests(f.spans.Spans, f.reverse, f.lockStrength, f.reqsScratch) + ba.Requests = spansToRequests(f.spans.Spans, f.scanFormat, f.reverse, f.lockStrength, f.reqsScratch) if log.ExpensiveLogEnabled(ctx, 2) { log.VEventf(ctx, 2, "Scan %s", f.spans) @@ -664,12 +687,16 @@ const requestUnionOverhead = int64(unsafe.Sizeof(roachpb.RequestUnion{})) // spansToRequests converts the provided spans to the corresponding requests. If // a span doesn't have the EndKey set, then a Get request is used for it; // otherwise, a Scan (or ReverseScan if reverse is true) request is used with -// BATCH_RESPONSE format. +// the provided scan format. // // The provided reqsScratch is reused if it has enough capacity for all spans, // if not, a new slice is allocated. func spansToRequests( - spans roachpb.Spans, reverse bool, keyLocking lock.Strength, reqsScratch []roachpb.RequestUnion, + spans roachpb.Spans, + scanFormat roachpb.ScanFormat, + reverse bool, + keyLocking lock.Strength, + reqsScratch []roachpb.RequestUnion, ) []roachpb.RequestUnion { var reqs []roachpb.RequestUnion if cap(reqsScratch) >= len(spans) { @@ -710,7 +737,7 @@ func spansToRequests( } curScan := i - curGet scans[curScan].req.SetSpan(spans[i]) - scans[curScan].req.ScanFormat = roachpb.BATCH_RESPONSE + scans[curScan].req.ScanFormat = scanFormat scans[curScan].req.KeyLocking = keyLocking scans[curScan].union.ReverseScan = &scans[curScan].req reqs[i].Value = &scans[curScan].union @@ -733,7 +760,7 @@ func spansToRequests( } curScan := i - curGet scans[curScan].req.SetSpan(spans[i]) - scans[curScan].req.ScanFormat = roachpb.BATCH_RESPONSE + scans[curScan].req.ScanFormat = scanFormat scans[curScan].req.KeyLocking = keyLocking scans[curScan].union.Scan = &scans[curScan].req reqs[i].Value = &scans[curScan].union diff --git a/pkg/sql/row/kv_batch_streamer.go b/pkg/sql/row/kv_batch_streamer.go index ac5ac4868906..cfe0dd42b858 100644 --- a/pkg/sql/row/kv_batch_streamer.go +++ b/pkg/sql/row/kv_batch_streamer.go @@ -57,7 +57,7 @@ func newTxnKVStreamer( ) KVBatchFetcher { f := &txnKVStreamer{ streamer: streamer, - keyLocking: getKeyLockingStrength(lockStrength), + keyLocking: GetKeyLockingStrength(lockStrength), acc: acc, } f.kvBatchFetcherHelper.init(f.nextBatch, batchRequestsIssued) @@ -95,7 +95,8 @@ func (f *txnKVStreamer) SetupNextFetch( for i := len(spans); i < len(reqsScratch); i++ { reqsScratch[i] = roachpb.RequestUnion{} } - reqs := spansToRequests(spans, false /* reverse */, f.keyLocking, reqsScratch) + // TODO(yuzefovich): consider supporting COL_BATCH_RESPONSE scan format. + reqs := spansToRequests(spans, roachpb.BATCH_RESPONSE, false /* reverse */, f.keyLocking, reqsScratch) if err := f.streamer.Enqueue(ctx, reqs); err != nil { return err } diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 847f813b71af..3db892530386 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -33,6 +34,7 @@ import ( type KVFetcher struct { KVBatchFetcher + kv roachpb.KeyValue kvs []roachpb.KeyValue batchResponse []byte @@ -105,6 +107,34 @@ func newTxnKVFetcher( return newTxnKVFetcherInternal(fetcherArgs) } +// NewDirectKVBatchFetcher creates a new KVBatchFetcher that uses the +// COL_BATCH_RESPONSE scan format for Scans (or ReverseScans, if reverse is +// true). +// +// If acc is non-nil, this fetcher will track its fetches and must be Closed. +// The fetcher only grows and shrinks the account according to its own use, so +// the memory account can be shared by the caller with other components (as long +// as there is no concurrency). +func NewDirectKVBatchFetcher( + txn *kv.Txn, + bsHeader *roachpb.BoundedStalenessHeader, + spec *fetchpb.IndexFetchSpec, + reverse bool, + lockStrength descpb.ScanLockingStrength, + lockWaitPolicy descpb.ScanLockingWaitPolicy, + lockTimeout time.Duration, + acc *mon.BoundAccount, + forceProductionKVBatchSize bool, +) KVBatchFetcher { + f := newTxnKVFetcher( + txn, bsHeader, reverse, lockStrength, lockWaitPolicy, + lockTimeout, acc, forceProductionKVBatchSize, + ) + f.scanFormat = roachpb.COL_BATCH_RESPONSE + f.indexFetchSpec = spec + return f +} + // NewKVFetcher creates a new KVFetcher. // // If acc is non-nil, this fetcher will track its fetches and must be Closed. @@ -156,7 +186,7 @@ func NewStreamingKVFetcher( streamerBudgetLimit, streamerBudgetAcc, &batchRequestsIssued, - getKeyLockingStrength(lockStrength), + GetKeyLockingStrength(lockStrength), ) mode := kvstreamer.OutOfOrder if maintainOrdering { @@ -183,15 +213,11 @@ func newKVFetcher(batchFetcher KVBatchFetcher) *KVFetcher { // that generated this kv (0 if nil spanIDs were provided when constructing the // fetcher), and any errors that may have occurred. // -// needsCopy is set to true when the caller should copy the returned KeyValue. -// One example of when this happens is when the returned KV's byte slices are -// the last reference into a larger backing byte slice. In such a case, the next -// call to NextKV might potentially allocate a big chunk of new memory, and by -// copying the returned KeyValue into a small slice that the caller owns, we -// avoid retaining two large backing byte slices at once. +// The returned kv is stable meaning that it will not be invalidated on the +// following nextKV call. func (f *KVFetcher) nextKV( ctx context.Context, mvccDecodeStrategy storage.MVCCDecodingStrategy, -) (ok bool, kv roachpb.KeyValue, spanID int, needsCopy bool, err error) { +) (ok bool, kv roachpb.KeyValue, spanID int, err error) { for { // Only one of f.kvs or f.batchResponse will be set at a given time. Which // one is set depends on the format returned by a given BatchRequest. @@ -202,7 +228,7 @@ func (f *KVFetcher) nextKV( // We always return "false" for needsCopy when returning data in the // KV format, because each of the KVs doesn't share any backing memory - // they are all independently garbage collectable. - return true, kv, f.spanID, false, nil + return true, kv, f.spanID, nil } if len(f.batchResponse) > 0 { var key []byte @@ -216,26 +242,37 @@ func (f *KVFetcher) nextKV( key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) } if err != nil { - return false, kv, 0, false, err + return false, kv, 0, err + } + key = key[:len(key):len(key)] + rawBytes = rawBytes[:len(rawBytes):len(rawBytes)] + // By default, use the references to the key and the value directly. + f.kv = roachpb.KeyValue{ + Key: key, + Value: roachpb.Value{ + RawBytes: rawBytes, + Timestamp: ts, + }, } // If we're finished decoding the batch response, nil our reference to it // so that the garbage collector can reclaim the backing memory. lastKey := len(f.batchResponse) == 0 if lastKey { f.batchResponse = nil + // If we've made it to the very last key in the batch, copy out + // the key so that the GC can reclaim the large backing slice + // before nextKV() is called again. + f.kv.Key = make(roachpb.Key, len(key)) + copy(f.kv.Key, key) + f.kv.Value.RawBytes = make([]byte, len(rawBytes)) + copy(f.kv.Value.RawBytes, rawBytes) } - return true, roachpb.KeyValue{ - Key: key[:len(key):len(key)], - Value: roachpb.Value{ - RawBytes: rawBytes[:len(rawBytes):len(rawBytes)], - Timestamp: ts, - }, - }, f.spanID, lastKey, nil + return true, f.kv, f.spanID, nil } resp, err := f.NextBatch(ctx) if err != nil || !resp.MoreKVs { - return resp.MoreKVs, roachpb.KeyValue{}, 0, false, err + return resp.MoreKVs, roachpb.KeyValue{}, 0, err } f.kvs = resp.KVs f.batchResponse = resp.BatchResponse @@ -243,18 +280,27 @@ func (f *KVFetcher) nextKV( } } +// Init implements the storage.NextKVer interface. +func (f *KVFetcher) Init(storage.FirstKeyOfRowGetter) (stableKVs bool) { + // nextKV never invalidates the returned kv, so it is always stable. + return true +} + // NextKV implements the storage.NextKVer interface. // gcassert:inline func (f *KVFetcher) NextKV( ctx context.Context, mvccDecodeStrategy storage.MVCCDecodingStrategy, -) (ok bool, kv roachpb.KeyValue, needsCopy bool, err error) { - ok, kv, _, needsCopy, err = f.nextKV(ctx, mvccDecodeStrategy) - return ok, kv, needsCopy, err -} - -// GetLastEncodedKey implements the storage.NextKVer interface. -func (f *KVFetcher) GetLastEncodedKey() roachpb.Key { - panic("unimplemented") +) (ok bool, partialRow bool, kv roachpb.KeyValue, err error) { + ok, kv, _, err = f.nextKV(ctx, mvccDecodeStrategy) + // nextKV never splits rows. + // + // Generally speaking, this is _not_ achieved via the WholeRowsOfSize option + // (although that option is used the txnKVStreamer). Instead, if one + // BatchResponse stops in the middle of a SQL row, then a follow-up + // BatchRequest is issued with the corresponding ResumeSpan, so nextKV() + // provides a stream of KVs that never stops in the middle of a SQL row. + partialRow = false + return ok, partialRow, kv, err } // SetupNextFetch overrides the same method from the wrapped KVBatchFetcher in diff --git a/pkg/sql/row/locking.go b/pkg/sql/row/locking.go index 657343048be9..ad9f17843c28 100644 --- a/pkg/sql/row/locking.go +++ b/pkg/sql/row/locking.go @@ -16,9 +16,9 @@ import ( "github.com/cockroachdb/errors" ) -// getKeyLockingStrength returns the configured per-key locking strength to use +// GetKeyLockingStrength returns the configured per-key locking strength to use // for key-value scans. -func getKeyLockingStrength(lockStrength descpb.ScanLockingStrength) lock.Strength { +func GetKeyLockingStrength(lockStrength descpb.ScanLockingStrength) lock.Strength { switch lockStrength { case descpb.ScanLockingStrength_FOR_NONE: return lock.None diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto index 73c6f030cfa1..48004803a16f 100644 --- a/pkg/sql/sessiondatapb/session_data.proto +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -104,6 +104,10 @@ message SessionData { // ColIndexJoin operator (when it is using the Streamer API) to construct a // single lookup KV batch. int64 index_join_streamer_batch_size = 24; + // DirectColumnarScansEnabled indicates whether the COL_BATCH_RESPONSE scan + // format should be used for ScanRequests and ReverseScanRequests whenever + // possible. + bool direct_columnar_scans_enabled = 25; } // DataConversionConfig contains the parameters that influence the output diff --git a/pkg/sql/trace_test.go b/pkg/sql/trace_test.go index c0ca54ad75a1..1cd577f82561 100644 --- a/pkg/sql/trace_test.go +++ b/pkg/sql/trace_test.go @@ -244,6 +244,11 @@ func TestTrace(t *testing.T) { if _, err := sqlDB.Exec("SET vectorize = on"); err != nil { t.Fatal(err) } + // Disable the direct columnar scans to make the vectorized + // planning deterministic. + if _, err := sqlDB.Exec(`SET direct_columnar_scans_enabled = false`); err != nil { + t.Fatal(err) + } if _, err := sqlDB.Exec("SET tracing = on; SELECT * FROM test.foo; SET tracing = off"); err != nil { t.Fatal(err) } diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 3b0838ddbd15..f0648b65938b 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -472,6 +472,25 @@ var varGen = map[string]sessionVar{ GlobalDefault: globalFalse, }, + // CockroachDB extension. + `direct_columnar_scans_enabled`: { + GetStringVal: makePostgresBoolGetStringValFn(`direct_columnar_scans_enabled`), + Set: func(_ context.Context, m sessionDataMutator, s string) error { + b, err := paramparse.ParseBoolVar(`direct_columnar_scans_enabled`, s) + if err != nil { + return err + } + m.SetDirectColumnarScansEnabled(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { + return formatBoolAsPostgresSetting(evalCtx.SessionData().DirectColumnarScansEnabled), nil + }, + GlobalDefault: func(sv *settings.Values) string { + return formatBoolAsPostgresSetting(colfetcher.DirectScansEnabled.Get(sv)) + }, + }, + // CockroachDB extension. `disable_plan_gists`: { GetStringVal: makePostgresBoolGetStringValFn(`disable_plan_gists`), diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 76f3a64b7e55..d5a383a7ce68 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -56,6 +56,7 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/catalog/fetchpb", "//pkg/storage/enginepb", "//pkg/storage/fs", "//pkg/storage/pebbleiter", diff --git a/pkg/storage/col_mvcc.go b/pkg/storage/col_mvcc.go index 687534f16554..67f5e666ccaa 100644 --- a/pkg/storage/col_mvcc.go +++ b/pkg/storage/col_mvcc.go @@ -11,31 +11,471 @@ package storage import ( + "bytes" "context" + "math" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/fetchpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/errors" ) +// This file defines several interfaces as well as introduces a couple of +// components that power the direct columnar scans. The main idea of this +// feature is to use the injected decoding logic from SQL in order to process +// each KV and keep only the needed parts (i.e. necessary SQL columns). Those +// needed parts are then propagated back to the KV client as coldata.Batch'es +// (serialized in the Apache Arrow format). +// +// Here is an example outline of all components involved: +// +// ┌────────────────────────────────────────────────┐ +// │ SQL │ +// │________________________________________________│ +// │ colfetcher.ColBatchDirectScan │ +// │ │ │ +// │ ▼ │ +// │ row.txnKVFetcher │ +// │ (behind the row.KVBatchFetcher interface) │ +// └────────────────────────────────────────────────┘ +// │ +// ▼ +// ┌────────────────────────────────────────────────┐ +// │ KV Client │ +// └────────────────────────────────────────────────┘ +// │ +// ▼ +// ┌────────────────────────────────────────────────┐ +// │ KV Server │ +// │________________________________________________│ +// │ colfetcher.cFetcherWrapper │ +// │ (behind the storage.CFetcherWrapper interface) │ +// │ │ │ +// │ ▼ │ +// │ colfetcher.cFetcher │ +// │ │ │ +// │ ▼ │ +// │ storage.mvccScanFetchAdapter ────────┐│ +// │ (behind the storage.NextKVer interface) ││ +// │ │ ││ +// │ ▼ ││ +// │ storage.pebbleMVCCScanner ││ +// │ (which put's KVs into storage.singleResults) <┘│ +// └────────────────────────────────────────────────┘ +// +// On the KV client side, row.txnKVFetcher issues Scans and ReverseScans with +// the COL_BATCH_RESPONSE format and returns the response (which contains the +// columnar data) to the colfetcher.ColBatchDirectScan. +// +// On the KV server side, we create a storage.CFetcherWrapper that asks the +// colfetcher.cFetcher for the next coldata.Batch. The cFetcher, in turn, +// fetches the next KV, decodes it, and keeps only values for the needed SQL +// columns, discarding the rest of the KV. The KV is emitted by the +// mvccScanFetchAdapter which - via the singleResults struct - exposes access to +// the current KV that the pebbleMVCCScanner is pointing at. +// +// Note that there is an additional "implicit synchronization" between +// components that is not shown on this diagram. In particular, +// storage.singleResults.maybeTrimPartialLastRow must be in sync with the +// colfetcher.cFetcher which is achieved by +// - the cFetcher exposing access to the first key of the last incomplete SQL +// row via the FirstKeyOfRowGetter, +// - the singleResults using that key as the resume key for the response, +// - and the cFetcher removing that last partial SQL row when NextKV() returns +// partialRow=true. +// This "upstream" link (although breaking the layering a bit) allows us to +// avoid a performance penalty for handling the case with multiple column +// families. (This case is handled by the storage.pebbleResults via tracking +// offsets into the pebbleResults.repr.) +// +// This code structure deserves some elaboration. First, there is a mismatch +// between the "push" mode in which the pebbleMVCCScanner operates and the +// "pull" mode that the NextKVer exposes. The adaption between two different +// modes is achieved via the mvccScanFetcherAdapter grabbing (when the control +// returns to it) the current unstable KV pair from the singleResults struct +// which serves as a one KV pair buffer that the pebbleMVCCScanner `put`s into. +// Second, in order be able to use the unstable KV pair without performing a +// copy, the pebbleMVCCScanner stops at the current KV pair and returns the +// control flow (which is exactly what pebbleMVCCScanner.getOne does) back to +// the mvccScanFetcherAdapter, with the adapter advancing the scanner only when +// the next KV pair is needed. + +// FirstKeyOfRowGetter returns the first key included into the last incomplete +// SQL row by the user of NextKVer. If the last row is complete, then nil is +// returned. +type FirstKeyOfRowGetter func() roachpb.Key + // NextKVer can fetch a new KV from somewhere. If MVCCDecodingStrategy is set // to MVCCDecodingRequired, the returned KV will include a timestamp. type NextKVer interface { - // NextKV returns the next kv from this NextKVer. Returns false if there are - // no more kvs to fetch, the kv that was fetched, and any errors that may - // have occurred. + // Init initializes the NextKVer. It returns a boolean indicating whether + // the KVs returned by NextKV are stable (i.e. whether they will not be + // invalidated by calling NextKV again). + Init(getter FirstKeyOfRowGetter) (stableKVs bool) + // NextKV returns the next kv from this NextKVer. + // - ok=false indicates that there are no more kvs to fetch, + // - partialRow indicates whether the fetch stopped in the middle of a SQL + // row (in this case ok will be set to false), + // - the kv that was fetched, + // - any errors that may have occurred. // - // needsCopy is set to true when the caller should copy the returned - // KeyValue. One example of when this happens is when the returned KV's byte - // slices are the last reference into a larger backing byte slice. In such a - // case, the next call to NextKV might potentially allocate a big chunk of - // new memory, and by copying the returned KeyValue into a small slice that - // the caller owns, we avoid retaining two large backing byte slices at - // once. - NextKV(context.Context, MVCCDecodingStrategy) ( - ok bool, kv roachpb.KeyValue, needsCopy bool, err error, + // When (ok=false,partialRow=true) is returned, the caller is expected to + // discard all KVs that were part of the last SQL row that was incomplete. + // The scan will be resumed from the key provided by the FirstKeyOfRowGetter + // (provided in Init by the caller) obtained during this NextKV call. + NextKV(context.Context, MVCCDecodingStrategy) (ok bool, partialRow bool, kv roachpb.KeyValue, err error) +} + +// CFetcherWrapper is a wrapper around a colfetcher.cFetcher that populates only +// the needed (according to the fetchpb.IndexFetchSpec) vectors and returns a +// serialized representation of coldata.Batch. +type CFetcherWrapper interface { + // NextBatch gives back the next column-oriented batch, serialized in Arrow + // batch format. + NextBatch(ctx context.Context) ([]byte, error) + + // Close release the resources held by this CFetcherWrapper. It *must* be + // called after use of the wrapper. + Close(ctx context.Context) +} + +// GetCFetcherWrapper returns a CFetcherWrapper. It's injected from +// pkg/sql/colfetcher to avoid circular dependencies since storage can't depend +// on higher levels of the system. +var GetCFetcherWrapper func( + ctx context.Context, + fetcherAccount, converterAccount *mon.BoundAccount, + indexFetchSpec *fetchpb.IndexFetchSpec, + nextKVer NextKVer, + startKey roachpb.Key, +) (CFetcherWrapper, error) + +// onNextKVFn represents the transition that the mvccScanFetchAdapter needs to +// perform on the following NextKV() call. +type onNextKVFn int + +const ( + _ onNextKVFn = iota + // onNextKVSeek is the initial state of the mvccScanFetchAdapter where it + // must seek to the start of the scan. The state machine will then + // transition to the onNextKVAdvance state. + onNextKVSeek + // onNextKVAdvance is the main state of the mvccScanFetchAdapter where it + // advances the scanner to the next KV (which is then returned on the NextKV + // call). Once there are no more KVs to scan (either because the scan was + // exhausted or some kind of limit was reached), the state machine will + // transition to the onNextKVDone state. + onNextKVAdvance + // onNextKVDone is the final state of the mvccScanFetchAdapter which + // indicates that the current scan is complete. + onNextKVDone +) + +// mvccScanFetchAdapter is a NextKVer that is powered directly by the +// pebbleMVCCScanner. Each time its NextKV is called, it advances the pebble +// iterator and returns a single KV. Note that the returned KV is only valid +// until the next call to NextKV. +type mvccScanFetchAdapter struct { + scanner *pebbleMVCCScanner + machine onNextKVFn + results singleResults +} + +var _ NextKVer = &mvccScanFetchAdapter{} + +// Init implements the NextKVer interface. +func (f *mvccScanFetchAdapter) Init(firstKeyGetter FirstKeyOfRowGetter) (stableKVs bool) { + f.results.firstKeyGetter = firstKeyGetter + // The returned kv is never stable because it'll be invalidated by the + // pebbleMVCCScanner on each NextKV() call. + return false +} + +// NextKV implements the NextKVer interface. +func (f *mvccScanFetchAdapter) NextKV( + ctx context.Context, mvccDecodingStrategy MVCCDecodingStrategy, +) (ok bool, partialRow bool, kv roachpb.KeyValue, err error) { + // Perform the action according to the current state. + switch f.machine { + case onNextKVSeek: + if !f.scanner.seekToStartOfScan() { + return false, false, roachpb.KeyValue{}, f.scanner.err + } + f.machine = onNextKVAdvance + case onNextKVAdvance: + if !f.scanner.advance() { + // No more keys in the scan. + return false, false, roachpb.KeyValue{}, nil + } + case onNextKVDone: + // No more keys in the scan. + return false, f.results.partialRowTrimmed, roachpb.KeyValue{}, nil + } + // Attempt to get one KV. + ok, added := f.scanner.getOne(ctx) + if !ok { + // ok=false indicates that the iteration must stop, so we're done after + // we process the current KV (if it was added). + f.machine = onNextKVDone + } + if !added { + // The KV wasn't added for whatever reason (e.g. it could have been + // skipped over due to having been deleted), so just move on. + return f.NextKV(ctx, mvccDecodingStrategy) + } + // We have a KV to return. Decode it according to mvccDecodingStrategy. + kv = f.results.getLastKV() + mvccKey := kv.Key + if buildutil.CrdbTestBuild { + if len(mvccKey) == 0 || len(kv.Value.RawBytes) == 0 { + return false, false, kv, errors.AssertionFailedf("unexpectedly received an empty lastKV") + } + } + switch mvccDecodingStrategy { + case MVCCDecodingRequired: + kv.Key, kv.Value.Timestamp, err = enginepb.DecodeKey(mvccKey) + if err != nil { + return false, false, kv, errors.AssertionFailedf("invalid encoded mvcc key: %x", mvccKey) + } + case MVCCDecodingNotRequired: + kv.Key, _, ok = enginepb.SplitMVCCKey(mvccKey) + if !ok { + return false, false, kv, errors.AssertionFailedf("invalid encoded mvcc key: %x", mvccKey) + } + } + return true, false, kv, nil +} + +// singleResults is an implementation of the results interface that is able to +// hold only a single KV at a time - all KVs are "accumulated" in the +// colfetcher.cFetcher. +// +// Here is how all things fit together: +// - the colfetcher.cFetcher calls NextKV on the mvccScanFetchAdapter; +// - the mvccScanFetchAdapter advances the pebbleMVCCScanner to the next key; +// - the mvccScanFetchAdapter asks the scanner to `getOne` which `put`s a new +// KV into the `singleResults`. Importantly, the pebbleMVCCScanner is not +// eagerly advancing further which allows us to just use the unstable +// key-value from the pebbleMVCCScanner; +// - the mvccScanFetchAdapter peeks into the `singleResults` struct to extract +// the new KV, possibly decodes the timestamp, and returns it to the +// colfetcher.cFetcher for processing; +// - the colfetcher.cFetcher decodes the KV, and goes back to the first step. +type singleResults struct { + maxKeysPerRow uint32 + maxFamilyID uint32 + onClear func() + count, bytes int64 + mvccKey []byte + value []byte + firstKeyGetter FirstKeyOfRowGetter + // firstRowKeyPrefix is a deep copy of the "row prefix" of the first SQL row + // seen by the singleResults (only set when the table has multiple column + // families). + firstRowKeyPrefix []byte + partialRowTrimmed bool +} + +var _ results = &singleResults{} + +// clear implements the results interface. +func (s *singleResults) clear() { + s.onClear() + *s = singleResults{} +} + +func singleResultsKVSizeOf(lenKey, lenValue int) int64 { + // TODO(yuzefovich, 23.1): come up with a formula that better represents the + // footprint of the serialized batches. + return int64(lenKey + lenValue) +} + +// sizeInfo implements the results interface. +func (s *singleResults) sizeInfo(lenKey, lenValue int) (numKeys, numBytes, numBytesInc int64) { + numKeys = s.count + // TODO(yuzefovich, 23.1): consider using the footprint of coldata.Batches + // so far (or of serialized representations) here. + numBytes = s.bytes + numBytesInc = singleResultsKVSizeOf(lenKey, lenValue) + return numKeys, numBytes, numBytesInc +} + +// put implements the results interface. +func (s *singleResults) put( + ctx context.Context, mvccKey []byte, value []byte, memAccount *mon.BoundAccount, _ int, +) error { + bytesInc := singleResultsKVSizeOf(len(mvccKey), len(value)) + if err := memAccount.Grow(ctx, bytesInc); err != nil { + return err + } + s.count++ + s.bytes += bytesInc + s.mvccKey = mvccKey + s.value = value + if s.count == 1 && s.maxKeysPerRow > 1 { + // If this is the first key, and we have multiple column families, then + // we store the deep-copied row prefix of this key. This is needed to + // implement continuesFirstRow. + key, _, ok := enginepb.SplitMVCCKey(mvccKey) + if !ok { + return errors.AssertionFailedf("invalid encoded mvcc key: %x", mvccKey) + } + firstRowKeyPrefix := getRowPrefix(key) + s.firstRowKeyPrefix = make([]byte, len(firstRowKeyPrefix)) + copy(s.firstRowKeyPrefix, firstRowKeyPrefix) + } + return nil +} + +// continuesFirstRow implements the results interface. +func (s *singleResults) continuesFirstRow(key roachpb.Key) bool { + rowPrefix := getRowPrefix(key) + if rowPrefix == nil { + return false + } + return bytes.Equal(rowPrefix, s.firstRowKeyPrefix) +} + +// maybeTrimPartialLastRow implements the results interface. +func (s *singleResults) maybeTrimPartialLastRow(key roachpb.Key) (roachpb.Key, error) { + firstKeyOfRow := s.firstKeyGetter() + // getRowPrefix handles the case of empty key, so we don't need to check + // that explicitly upfront. + if !bytes.Equal(getRowPrefix(firstKeyOfRow), getRowPrefix(key)) { + // The given key is the first KV of the next row, so we will resume the + // scan from this key. + return key, nil + } + // The given key is part of the current last row, and it will be removed by + // the cFetcher (since NextKV() will return partialRow=true before the row + // can be completed), thus, we'll resume the scan from the first key in the + // last row. + s.partialRowTrimmed = true + return firstKeyOfRow, nil +} + +// lastRowHasFinalColumnFamily implements the results interface. +func (s *singleResults) lastRowHasFinalColumnFamily(reverse bool) bool { + key, _, ok := enginepb.SplitMVCCKey(s.mvccKey) + if !ok { + return false + } + return keyHasFinalColumnFamily(key, s.maxFamilyID, reverse) +} + +func (s *singleResults) getLastKV() roachpb.KeyValue { + return roachpb.KeyValue{ + Key: s.mvccKey, + Value: roachpb.Value{RawBytes: s.value}, + } +} + +// MVCCScanToCols is like MVCCScan, but it returns KVData in a serialized +// columnar batch suitable for reading by colserde.RecordBatchDeserializer. +func MVCCScanToCols( + ctx context.Context, + reader Reader, + indexFetchSpec *fetchpb.IndexFetchSpec, + key, endKey roachpb.Key, + timestamp hlc.Timestamp, + opts MVCCScanOptions, + st *cluster.Settings, +) (MVCCScanResult, error) { + iter := newMVCCIterator( + reader, timestamp, !opts.Tombstones, opts.DontInterleaveIntents, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: key, + UpperBound: endKey, + }, + ) + defer iter.Close() + return mvccScanToCols(ctx, iter, indexFetchSpec, key, endKey, timestamp, opts, st) +} + +func mvccScanToCols( + ctx context.Context, + iter MVCCIterator, + indexFetchSpec *fetchpb.IndexFetchSpec, + key, endKey roachpb.Key, + timestamp hlc.Timestamp, + opts MVCCScanOptions, + st *cluster.Settings, +) (MVCCScanResult, error) { + adapter := mvccScanFetchAdapter{machine: onNextKVSeek} + adapter.results.maxKeysPerRow = indexFetchSpec.MaxKeysPerRow + adapter.results.maxFamilyID = uint32(indexFetchSpec.MaxFamilyID) + ok, mvccScanner, res, err := mvccScanInit(iter, key, endKey, timestamp, opts, &adapter.results) + if !ok { + return res, err + } + defer mvccScanner.release() + adapter.scanner = mvccScanner + + // Try to use the same root monitor (from the store) if the account is + // provided. + monitor := opts.MemoryAccount.Monitor() + if monitor == nil { + // If we don't have the monitor, then we create a "fake" one that is not + // connected to the memory accounting system. + monitor = mon.NewMonitor( + "mvcc-scan-to-cols", + mon.MemoryResource, + nil, /* curCount */ + nil, /* maxHist */ + -1, /* increment */ + math.MaxInt64, /* noteworthy */ + st, + ) + monitor.Start(ctx, nil /* pool */, mon.NewStandaloneBudget(math.MaxInt64)) + defer monitor.Stop(ctx) + } + fetcherAcc, converterAcc := monitor.MakeBoundAccount(), monitor.MakeBoundAccount() + defer fetcherAcc.Close(ctx) + defer converterAcc.Close(ctx) + wrapper, err := GetCFetcherWrapper( + ctx, + &fetcherAcc, + &converterAcc, + indexFetchSpec, + &adapter, + key, ) + if err != nil { + return MVCCScanResult{}, err + } + defer wrapper.Close(ctx) + + adapter.results.onClear = func() { + // Discard the accumulated batches on results.clear() call - the scan + // will result in an error. + res = MVCCScanResult{} + } + for { + batch, err := wrapper.NextBatch(ctx) + if err != nil { + return res, err + } + if batch == nil { + break + } + // We need to make a copy since the wrapper reuses underlying bytes + // buffer. + b := make([]byte, len(batch)) + copy(b, batch) + res.KVData = append(res.KVData, b) + } - // GetLastEncodedKey returns the key that was returned on the last NextKV() - // call. This method allows callers to access the key at different layers of - // abstraction. - GetLastEncodedKey() roachpb.Key + res.ResumeSpan, res.ResumeReason, res.ResumeNextBytes, err = mvccScanner.afterScan() + if err != nil { + return MVCCScanResult{}, err + } + if err = finalizeScanResult(ctx, mvccScanner, &res, opts.errOnIntents()); err != nil { + return MVCCScanResult{}, err + } + return res, nil } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 6491aa6a28dd..53803ed89e81 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -3645,34 +3645,41 @@ func recordIteratorStats(ctx context.Context, iter MVCCIterator) { }) } -func mvccScanToBytes( - ctx context.Context, +// mvccScanInit performs some preliminary checks on the validity of options for +// a scan. +// +// If ok=true is returned, then the pebbleMVCCScanner must be release()'d when +// no longer needed. The scanner is initialized with the given results. +// +// If ok=false is returned, then the returned result and the error are the +// result of the scan. +func mvccScanInit( iter MVCCIterator, key, endKey roachpb.Key, timestamp hlc.Timestamp, opts MVCCScanOptions, -) (MVCCScanResult, error) { + results results, +) (ok bool, _ *pebbleMVCCScanner, _ MVCCScanResult, _ error) { if len(endKey) == 0 { - return MVCCScanResult{}, emptyKeyError() + return false, nil, MVCCScanResult{}, emptyKeyError() } if err := opts.validate(); err != nil { - return MVCCScanResult{}, err + return false, nil, MVCCScanResult{}, err } if opts.MaxKeys < 0 { - return MVCCScanResult{ + return false, nil, MVCCScanResult{ ResumeSpan: &roachpb.Span{Key: key, EndKey: endKey}, ResumeReason: roachpb.RESUME_KEY_LIMIT, }, nil } if opts.TargetBytes < 0 { - return MVCCScanResult{ + return false, nil, MVCCScanResult{ ResumeSpan: &roachpb.Span{Key: key, EndKey: endKey}, ResumeReason: roachpb.RESUME_BYTE_LIMIT, }, nil } mvccScanner := pebbleMVCCScannerPool.Get().(*pebbleMVCCScanner) - defer mvccScanner.release() *mvccScanner = pebbleMVCCScanner{ parent: iter, @@ -3694,15 +3701,28 @@ func mvccScanToBytes( keyBuf: mvccScanner.keyBuf, } + mvccScanner.init(opts.Txn, opts.Uncertainty, results) + return true /* ok */, mvccScanner, MVCCScanResult{}, nil +} + +func mvccScanToBytes( + ctx context.Context, + iter MVCCIterator, + key, endKey roachpb.Key, + timestamp hlc.Timestamp, + opts MVCCScanOptions, +) (MVCCScanResult, error) { var results pebbleResults if opts.WholeRowsOfSize > 1 { results.lastOffsetsEnabled = true results.lastOffsets = make([]int, opts.WholeRowsOfSize) } - mvccScanner.init(opts.Txn, opts.Uncertainty, &results) + ok, mvccScanner, res, err := mvccScanInit(iter, key, endKey, timestamp, opts, &results) + if !ok { + return res, err + } + defer mvccScanner.release() - var res MVCCScanResult - var err error res.ResumeSpan, res.ResumeReason, res.ResumeNextBytes, err = mvccScanner.scan(ctx) if err != nil { @@ -3710,20 +3730,33 @@ func mvccScanToBytes( } res.KVData = results.finish() - res.NumKeys, res.NumBytes, _ = results.sizeInfo(0 /* lenKey */, 0 /* lenValue */) + if err = finalizeScanResult(ctx, mvccScanner, &res, opts.errOnIntents()); err != nil { + return MVCCScanResult{}, err + } + return res, nil +} + +// finalizeScanResult updates the MVCCScanResult in-place after the scan was +// completed successfully. It also performs some additional auxiliary tasks +// (like recording iterators stats). +func finalizeScanResult( + ctx context.Context, mvccScanner *pebbleMVCCScanner, res *MVCCScanResult, errOnIntents bool, +) error { + res.NumKeys, res.NumBytes, _ = mvccScanner.results.sizeInfo(0 /* lenKey */, 0 /* lenValue */) // If we have a trace, emit the scan stats that we produced. recordIteratorStats(ctx, mvccScanner.parent) + var err error res.Intents, err = buildScanIntents(mvccScanner.intentsRepr()) if err != nil { - return MVCCScanResult{}, err + return err } - if opts.errOnIntents() && len(res.Intents) > 0 { - return MVCCScanResult{}, &roachpb.WriteIntentError{Intents: res.Intents} + if errOnIntents && len(res.Intents) > 0 { + return &roachpb.WriteIntentError{Intents: res.Intents} } - return res, nil + return nil } // mvccScanToKvs converts the raw key/value pairs returned by MVCCIterator.MVCCScan diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 49c85deb0131..9d5bb8e3d9f9 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -66,14 +66,35 @@ type results interface { sizeInfo(lenKey, lenValue int) (numKeys, numBytes, numBytesInc int64) // put adds a KV into the results. An error is returned if the memory // reservation is denied by the memory account. - put(_ context.Context, key []byte, value []byte, memAccount *mon.BoundAccount, maxNewSize int) error + put(_ context.Context, mvccKey []byte, value []byte, memAccount *mon.BoundAccount, maxNewSize int) error // continuesFirstRow returns true if the given key belongs to the same SQL // row as the first KV pair in the result (or if the result is empty). If - // either key is not a valid SQL row key, returns false. + // the given key is not a valid SQL row key, returns false. + // + // pebbleMVCCScanner.getOne can call this method only _before_ calling + // put(). This constraint ensures that for the singleResults implementation + // of this interface, when this method is called, there is no buffered KV + // (i.e. there is no KV that has been `put` into the results but not yet + // returned on the NextKVer.NextKV call). Therefore, the singleResults can + // make a determination on its own whether the given key belongs to the + // first SQL row. + // + // Only called when wholeRows option is enabled. continuesFirstRow(key roachpb.Key) bool // maybeTrimPartialLastRow removes the last KV pairs from the result that // are part of the same SQL row as the given key, returning the earliest key // removed. + // + // pebbleMVCCScanner.getOne can call this method only _before_ calling + // put(). This constraint ensures that for the singleResults implementation + // of this interface, when this method is called, there is no buffered KV + // (i.e. there is no KV that has been `put` into the results but not yet + // returned on the NextKVer.NextKV call). This allows for the singleResults + // to synchronize with the colfetcher.cFetcher (via the + // storage.FirstKeyOfRowGetter) to obtain the first key of the SQL row if + // the given key belongs to that row. + // + // Only called when wholeRows option is enabled. maybeTrimPartialLastRow(key roachpb.Key) (roachpb.Key, error) // lastRowHasFinalColumnFamily returns true if the last key in the result is // the maximum column family ID of the row. If so, we know that the row is @@ -81,9 +102,10 @@ type results interface { // the row may be omitted, in which case the caller has to scan to the next // key to find out whether the row is complete. // - // lastRowHasFinalColumnFamily is only called after having called put() with - // no error at least once, meaning that at least one key is in the results. - // Also, this is only called when wholeRows option is enabled. + // This method is called _after_ having called put() with no error at least + // once, meaning that at least one key is in the results. + // + // Only called when wholeRows option is enabled. lastRowHasFinalColumnFamily(reverse bool) bool } @@ -247,6 +269,19 @@ func (p *pebbleResults) continuesFirstRow(key roachpb.Key) bool { return bytes.Equal(rowPrefix, getRowPrefix(extractResultKey(repr))) } +// keyHasFinalColumnFamily returns whether the given key corresponds to the last +// column family in a SQL row. Returns false if the key is not a valid SQL key. +func keyHasFinalColumnFamily(key roachpb.Key, maxFamilyID uint32, reverse bool) bool { + colFamilyID, err := keys.DecodeFamilyKey(key) + if err != nil { + return false + } + if reverse { + return colFamilyID == 0 + } + return colFamilyID == maxFamilyID +} + // lastRowHasFinalColumnFamily implements the results interface. func (p *pebbleResults) lastRowHasFinalColumnFamily(reverse bool) bool { lastOffsetIdx := p.lastOffsetIdx - 1 // p.lastOffsetIdx is where next offset would be stored @@ -254,16 +289,8 @@ func (p *pebbleResults) lastRowHasFinalColumnFamily(reverse bool) bool { lastOffsetIdx = len(p.lastOffsets) - 1 } lastOffset := p.lastOffsets[lastOffsetIdx] - key := extractResultKey(p.repr[lastOffset:]) - colFamilyID, err := keys.DecodeFamilyKey(key) - if err != nil { - return false - } - if reverse { - return colFamilyID == 0 - } - return int(colFamilyID) == len(p.lastOffsets)-1 + return keyHasFinalColumnFamily(key, uint32(len(p.lastOffsets)-1), reverse) } // maybeTrimPartialLastRow implements the results interface. @@ -540,6 +567,24 @@ func (p *pebbleMVCCScanner) get(ctx context.Context) { } } +// seekToStartOfScan positions the scanner at the initial key. +func (p *pebbleMVCCScanner) seekToStartOfScan() (ok bool) { + if p.reverse { + if !p.iterSeekReverse(MVCCKey{Key: p.end}) { + p.maybeFailOnMoreRecent() // may have seen a conflicting range key + return false + } + p.machine.fn = advanceKeyReverse + } else { + if !p.iterSeek(MVCCKey{Key: p.start}) { + p.maybeFailOnMoreRecent() // may have seen a conflicting range key + return false + } + p.machine.fn = advanceKeyForward + } + return true +} + // advance advances the iterator according to the current state of the state // machine. func (p *pebbleMVCCScanner) advance() bool { @@ -577,27 +622,22 @@ func (p *pebbleMVCCScanner) scan( if p.wholeRows && !p.results.(*pebbleResults).lastOffsetsEnabled { return nil, 0, 0, errors.AssertionFailedf("cannot use wholeRows without trackLastOffsets") } - - if p.reverse { - if !p.iterSeekReverse(MVCCKey{Key: p.end}) { - p.maybeFailOnMoreRecent() // may have seen a conflicting range key - return nil, 0, 0, p.err - } - p.machine.fn = advanceKeyReverse - } else { - if !p.iterSeek(MVCCKey{Key: p.start}) { - p.maybeFailOnMoreRecent() // may have seen a conflicting range key - return nil, 0, 0, p.err - } - p.machine.fn = advanceKeyForward + if !p.seekToStartOfScan() { + return nil, 0, 0, p.err } - for ok := true; ok; { ok, _ = p.getOne(ctx) if ok { ok = p.advance() } } + return p.afterScan() +} + +// afterScan checks whether some limit was exceeded during the scan, and if so, +// it returns a resume span, resume reason, and for targetBytes the size of the +// next result. +func (p *pebbleMVCCScanner) afterScan() (*roachpb.Span, roachpb.ResumeReason, int64, error) { p.maybeFailOnMoreRecent() if p.err != nil { @@ -709,7 +749,8 @@ func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) (ok bool) { return false } -// Get one tuple into the result set. +// Get one tuple into the result set. This method will make at most one +// 'results.put' call regardless of whether 'put' returns an error or not. // - ok indicates whether the iteration should continue. // - added indicates whether a tuple was included into the result set. // (ok=true, added=false) indicates that the current key was skipped for some @@ -1128,6 +1169,12 @@ func (p *pebbleMVCCScanner) advanceKeyAtNewKeyReverse() bool { return p.prevKey(p.machine.origKey) } +// IncludeStartKeyIntoErr wraps with the given error to include the provided +// start key of the scan as an additional detail. +func IncludeStartKeyIntoErr(startKey roachpb.Key, err error) error { + return errors.Wrapf(err, "scan with start key %s", startKey) +} + // Adds the specified key and value to the result set, excluding tombstones // unless p.tombstones is true. // - ok indicates whether the iteration should continue. This can be false @@ -1201,7 +1248,7 @@ func (p *pebbleMVCCScanner) add( // // For B we will never set maxNewSize. // For A, we may set maxNewSize, but we already know that - // p.targetBytes >= p.results.bytes + lenToAdd + // p.targetBytes >= numBytes + numBytesInc // so maxNewSize will be sufficient. var maxNewSize int if p.targetBytes > 0 && p.targetBytes > numBytes && !mustPutKey { @@ -1210,7 +1257,7 @@ func (p *pebbleMVCCScanner) add( maxNewSize = int(p.targetBytes - numBytes) } if err := p.results.put(ctx, rawKey, rawValue, p.memAccount, maxNewSize); err != nil { - p.err = errors.Wrapf(err, "scan with start key %s", p.start) + p.err = IncludeStartKeyIntoErr(p.start, err) return false, false } numKeys++ @@ -1770,7 +1817,7 @@ func (p *pebbleMVCCScanner) addRawIntent(ctx context.Context, key, value []byte) // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. if p.err = p.memAccount.Grow(ctx, int64(len(key)+len(value))); p.err != nil { - p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) + p.err = IncludeStartKeyIntoErr(p.start, p.err) return false } p.err = p.intents.Set(key, value, nil)