Skip to content

Commit

Permalink
sql: use the streamer for scans
Browse files Browse the repository at this point in the history
To be filled.

`tpchvec/perf`:
```
Q1:	before: 3.23s	after: 3.24s	 0.46%
Q2:	before: 3.47s	after: 3.38s	 -2.72%
Q3:	before: 2.57s	after: 2.62s	 1.99%
Q4:	before: 1.78s	after: 1.73s	 -2.96%
Q5:	before: 2.53s	after: 2.56s	 1.03%
Q6:	before: 4.80s	after: 4.86s	 1.22%
Q7:	before: 5.96s	after: 5.96s	 -0.01%
Q8:	before: 1.13s	after: 1.14s	 0.41%
Q9:	before: 5.43s	after: 5.82s	 7.16%
Q10:	before: 2.24s	after: 2.20s	 -1.78%
Q11:	before: 0.98s	after: 0.99s	 0.67%
Q12:	before: 4.69s	after: 4.88s	 3.99%
Q13:	before: 1.17s	after: 1.11s	 -4.73%
Q14:	before: 0.47s	after: 0.48s	 0.93%
Q15:	before: 2.56s	after: 2.64s	 2.90%
Q16:	before: 0.98s	after: 0.94s	 -3.63%
Q17:	before: 0.25s	after: 0.24s	 -2.01%
Q18:	before: 2.11s	after: 2.03s	 -3.69%
Q19:	before: 0.50s	after: 0.48s	 -3.58%
Q20:	before: 9.61s	after: 10.35s	 7.61%
Q21:	before: 5.22s	after: 5.28s	 1.27%
Q22:	before: 0.60s	after: 0.59s	 -0.87%
```

`multitenant/tpch` (single-tenant):
```
Q1:	before: 5.77s	after: 4.85s	 -15.87%
Q2:	before: 3.81s	after: 3.59s	 -5.62%
Q3:	before: 4.04s	after: 3.59s	 -11.18%
Q4:	before: 1.31s	after: 1.35s	 3.15%
Q5:	before: 2.53s	after: 2.63s	 4.13%
Q6:	before: 6.94s	after: 7.06s	 1.74%
Q7:	before: 6.63s	after: 6.96s	 4.93%
Q8:	before: 1.00s	after: 1.02s	 2.53%
Q9:	before: 8.21s	after: 8.40s	 2.30%
Q10:	before: 1.81s	after: 1.85s	 1.86%
Q11:	before: 0.96s	after: 1.00s	 4.81%
Q12:	before: 7.29s	after: 7.51s	 2.97%
Q13:	before: 2.15s	after: 1.93s	 -10.21%
Q14:	before: 0.68s	after: 0.67s	 -0.68%
Q15:	before: 3.68s	after: 3.76s	 2.02%
Q16:	before: 0.87s	after: 0.87s	 0.39%
Q17:	before: 0.25s	after: 0.24s	 -6.78%
Q18:	before: 3.83s	after: 2.92s	 -23.72%
Q19:	before: 0.48s	after: 0.43s	 -10.44%
Q20:	before: 16.15s	after: 16.55s	 2.44%
Q21:	before: 6.97s	after: 7.25s	 4.01%
Q22:	before: 0.64s	after: 0.62s	 -2.42%
```

`multitenant/tpch` (multi-tenant):
```
Q1:	before: 11.36s	after: 10.20s	 -10.23%
Q2:	before: 4.54s	after: 5.23s	 15.27%
Q3:	before: 9.28s	after: 9.85s	 6.07%
Q4:	before: 4.55s	after: 4.55s	 0.11%
Q5:	before: 10.39s	after: 10.41s	 0.12%
Q6:	before: 33.69s	after: 33.76s	 0.18%
Q7:	before: 23.85s	after: 24.21s	 1.49%
Q8:	before: 3.79s	after: 3.81s	 0.58%
Q9:	before: 28.28s	after: 28.31s	 0.11%
Q10:	before: 5.00s	after: 5.01s	 0.12%
Q11:	before: 2.42s	after: 2.41s	 -0.57%
Q12:	before: 34.92s	after: 35.04s	 0.34%
Q13:	before: 3.71s	after: 3.16s	 -14.90%
Q14:	before: 2.60s	after: 3.24s	 24.41%
Q15:	before: 16.96s	after: 16.92s	 -0.23%
Q16:	before: 1.65s	after: 1.78s	 8.05%
Q17:	before: 1.11s	after: 1.12s	 0.65%
Q18:	before: 16.40s	after: 17.17s	 4.69%
Q19:	before: 1.98s	after: 1.70s	 -14.20%
Q20:	before: 55.45s	after: 55.46s	 0.01%
Q21:	before: 24.76s	after: 24.77s	 0.03%
Q22:	before: 1.24s	after: 1.29s	 3.67%
```

Release note: None
  • Loading branch information
yuzefovich committed Jul 17, 2023
1 parent f1945d9 commit 798c6b1
Show file tree
Hide file tree
Showing 17 changed files with 158 additions and 44 deletions.
11 changes: 9 additions & 2 deletions pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -856,7 +856,7 @@ func NewColOperator(
// unlimited one. We also need another unlimited account for the
// KV fetcher.
accounts := args.MonitorRegistry.CreateUnlimitedMemAccounts(
ctx, flowCtx, "cfetcher" /* opName */, spec.ProcessorID, 2, /* numAccounts */
ctx, flowCtx, "cfetcher" /* opName */, spec.ProcessorID, 3, /* numAccounts */
)
estimatedRowCount := spec.EstimatedRowCount
var scanOp colfetcher.ScanOperator
Expand Down Expand Up @@ -932,9 +932,16 @@ func NewColOperator(
}
}
if scanOp == nil {
var streamerDiskMonitor *mon.BytesMonitor
if core.TableReader.MaintainOrdering {
streamerDiskMonitor = args.MonitorRegistry.CreateDiskMonitor(
ctx, flowCtx, "streamer" /* opName */, spec.ProcessorID,
)
}
scanOp, resultTypes, err = colfetcher.NewColBatchScan(
ctx, colmem.NewAllocator(ctx, accounts[0], factory), accounts[1],
flowCtx, spec.ProcessorID, core.TableReader, post, estimatedRowCount, args.TypeResolver,
accounts[2], flowCtx, spec.ProcessorID, core.TableReader, post,
estimatedRowCount, streamerDiskMonitor, args.TypeResolver,
)
if err != nil {
return r, err
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/colfetcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//pkg/col/typeconv",
"//pkg/keys",
"//pkg/kv",
"//pkg/kv/kvclient/kvstreamer",
"//pkg/kv/kvpb",
"//pkg/roachpb",
"//pkg/settings",
Expand Down
81 changes: 70 additions & 11 deletions pkg/sql/colfetcher/colbatch_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,12 @@ package colfetcher

import (
"context"
"math"
"sync"
"time"

"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvstreamer"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
Expand All @@ -26,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowcontainer"
"github.com/cockroachdb/cockroach/pkg/sql/rowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/mon"
Expand Down Expand Up @@ -174,6 +177,9 @@ func newColBatchScanBase(
}

s := colBatchScanBasePool.Get().(*colBatchScanBase)
// TODO(yuzefovich): consider sorting these spans when
// spec.MaintainOrdering==false and when streamer isn't used (similar to
// what we do for lookup/index joins.
s.Spans = spec.Spans
if !flowCtx.Local {
// Make a copy of the spans so that we could get the misplanned ranges
Expand Down Expand Up @@ -331,11 +337,13 @@ func NewColBatchScan(
ctx context.Context,
fetcherAllocator *colmem.Allocator,
kvFetcherMemAcc *mon.BoundAccount,
streamerBudgetAcc *mon.BoundAccount,
flowCtx *execinfra.FlowCtx,
processorID int32,
spec *execinfrapb.TableReaderSpec,
post *execinfrapb.PostProcessSpec,
estimatedRowCount uint64,
diskMonitor *mon.BytesMonitor,
typeResolver *descs.DistSQLTypeResolver,
) (*ColBatchScan, []*types.T, error) {
base, bsHeader, tableArgs, err := newColBatchScanBase(
Expand All @@ -344,19 +352,70 @@ func NewColBatchScan(
if err != nil {
return nil, nil, err
}
kvFetcher := row.NewKVFetcher(
flowCtx.Txn,
bsHeader,
spec.Reverse,
spec.LockingStrength,
spec.LockingWaitPolicy,
flowCtx.EvalCtx.SessionData().LockTimeout,
kvFetcherMemAcc,
flowCtx.EvalCtx.TestingKnobs.ForceProductionValues,
)

totalMemoryLimit := execinfra.GetWorkMemLimit(flowCtx)
cFetcherMemoryLimit := totalMemoryLimit

var useStreamer bool
txn := flowCtx.Txn
// TODO(yuzefovich): support reverse scans in the streamer.
if !spec.Reverse {
useStreamer, txn, err = flowCtx.UseStreamer()
if err != nil {
return nil, nil, err
}
}
var kvFetcher *row.KVFetcher
if useStreamer {
if streamerBudgetAcc == nil {
return nil, nil, errors.AssertionFailedf("streamer budget account is nil when the Streamer API is desired")
}
var diskBuffer kvstreamer.ResultDiskBuffer
if spec.MaintainOrdering {
if diskMonitor == nil {
return nil, nil, errors.AssertionFailedf("diskMonitor is nil when ordering needs to be maintained")
}
diskBuffer = rowcontainer.NewKVStreamerResultDiskBuffer(
flowCtx.Cfg.TempStorage, diskMonitor,
)
}
// Keep 1/16th of the memory limit for the output batch of the cFetcher,
// and we'll give the remaining memory to the streamer budget below.
cFetcherMemoryLimit = int64(math.Ceil(float64(totalMemoryLimit) / 16.0))
streamerBudgetLimit := 15 * cFetcherMemoryLimit
kvFetcher = row.NewStreamingKVFetcher(
flowCtx.Cfg.DistSender,
flowCtx.Stopper(),
txn,
flowCtx.EvalCtx.Settings,
spec.LockingWaitPolicy,
spec.LockingStrength,
streamerBudgetLimit,
streamerBudgetAcc,
spec.MaintainOrdering,
false, /* singleRowLookup */
int(spec.FetchSpec.MaxKeysPerRow),
diskBuffer,
kvFetcherMemAcc,
)
// TODO(XXX): think through this.
base.batchBytesLimit = 0
} else {
kvFetcher = row.NewKVFetcher(
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),
cFetcherMemoryLimit,
estimatedRowCount,
flowCtx.TraceKV,
true, /* singleUse */
Expand Down
48 changes: 24 additions & 24 deletions pkg/sql/colfetcher/index_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,13 +124,22 @@ type ColIndexJoin struct {
// table because the scan might synthesize additional implicit system columns.
ResultTypes []*types.T

// maintainOrdering is true when the index join is required to maintain its
// input ordering, in which case the ordering of the spans cannot be changed.
maintainOrdering bool

// usesStreamer indicates whether the ColIndexJoin is using the Streamer
// API.
usesStreamer bool

// sortSpans indicates whether the ColIndexJoin should sort the spans before
// passing them to the cFetcher.
//
// We do so when maintaining order is not required. This allows lower layers
// to optimize iteration over the data. Note that the looked up rows are
// output unchanged, in the retrieval order, so it is not safe to do this
// when maintainOrdering is true (the ordering to be maintained may be
// different than the ordering in the index).
//
// We don't want to sort the spans if we're using the Streamer since it will
// perform the sort on its own.
sortSpans bool
}

var _ ScanOperator = &ColIndexJoin{}
Expand Down Expand Up @@ -195,16 +204,7 @@ func (s *ColIndexJoin) Next() coldata.Batch {
s.state = indexJoinDone
continue
}

if !s.usesStreamer && !s.maintainOrdering {
// Sort the spans when !maintainOrdering. This allows lower layers to
// optimize iteration over the data. Note that the looked up rows are
// output unchanged, in the retrieval order, so it is not safe to do
// this when maintainOrdering is true (the ordering to be maintained
// may be different than the ordering in the index).
//
// We don't want to sort the spans here if we're using the
// Streamer since it will perform the sort on its own.
if s.sortSpans {
sort.Sort(spans)
}

Expand Down Expand Up @@ -609,16 +609,16 @@ func NewColIndexJoin(
)

op := &ColIndexJoin{
OneInputNode: colexecop.NewOneInputNode(input),
flowCtx: flowCtx,
processorID: processorID,
cf: fetcher,
spanAssembler: spanAssembler,
ResultTypes: tableArgs.typs,
maintainOrdering: spec.MaintainOrdering,
txn: txn,
usesStreamer: useStreamer,
limitHintHelper: execinfra.MakeLimitHintHelper(spec.LimitHint, post),
OneInputNode: colexecop.NewOneInputNode(input),
flowCtx: flowCtx,
processorID: processorID,
cf: fetcher,
spanAssembler: spanAssembler,
ResultTypes: tableArgs.typs,
txn: txn,
usesStreamer: useStreamer,
sortSpans: !spec.MaintainOrdering && !useStreamer,
limitHintHelper: execinfra.MakeLimitHintHelper(spec.LimitHint, post),
}
op.mem.inputBatchSizeLimit = getIndexJoinBatchSize(
useStreamer, flowCtx.EvalCtx.TestingKnobs.ForceProductionValues, flowCtx.EvalCtx.SessionData(),
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/distsql_physical_planner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1614,6 +1614,7 @@ func initTableReaderSpecTemplate(
TableDescriptorModificationTime: n.desc.GetModificationTime(),
LockingStrength: n.lockingStrength,
LockingWaitPolicy: n.lockingWaitPolicy,
MaintainOrdering: len(n.reqOrdering) > 0,
}
if err := rowenc.InitIndexFetchSpec(&s.FetchSpec, codec, n.desc, n.index, colIDs); err != nil {
return nil, execinfrapb.PostProcessSpec{}, err
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/distsql_running.go
Original file line number Diff line number Diff line change
Expand Up @@ -764,16 +764,24 @@ func (dsp *DistSQLPlanner) Run(
if p.Spec.Core.LocalPlanNode != nil {
return true
}
if p.Spec.Core.TableReader != nil {
if planCtx.planner != nil {
if aost := planCtx.planner.EvalContext().AsOfSystemTime; aost != nil &&
aost.BoundedStaleness {
// BoundedStaleness requires the RootTxn.
return true
}
}
}
}
return false
}()
if !containsNonDefaultLocking && !mustUseRootTxn {
if evalCtx.SessionData().StreamerEnabled {
for _, proc := range plan.Processors {
if jr := proc.Spec.Core.JoinReader; jr != nil {
// Both index and lookup joins, with and without
// ordering, are executed via the Streamer API that has
// concurrency.
if core := proc.Spec.Core; core.TableReader != nil || core.JoinReader != nil {
// TableReader and JoinReader cores might be powered
// by the Streamer API which has concurrency.
localState.HasConcurrency = true
break
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/distsql_spec_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,7 @@ func (e *distSQLSpecExecFactory) ConstructScan(
*trSpec = execinfrapb.TableReaderSpec{
Reverse: params.Reverse,
TableDescriptorModificationTime: tabDesc.GetModificationTime(),
MaintainOrdering: len(reqOrdering) > 0,
}
if err := rowenc.InitIndexFetchSpec(&trSpec.FetchSpec, e.planner.ExecCfg().Codec, tabDesc, idx, columnIDs); err != nil {
return nil, err
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/execinfra/readerbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -232,6 +232,7 @@ func (flowCtx *FlowCtx) UseStreamer() (bool, *kv.Txn, error) {
var UseStreamerEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.distsql.use_streamer.enabled",
// TODO(XXX): description update.
"determines whether the usage of the Streamer API is allowed. "+
"Enabling this will increase the speed of lookup/index joins "+
"while adhering to memory limits.",
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/execinfrapb/processors_sql.proto
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,8 @@ message TableReaderSpec {
// leaseholder of the beginning of the key spans to be scanned).
optional bool ignore_misplanned_ranges = 22 [(gogoproto.nullable) = false];

optional bool maintain_ordering = 23 [(gogoproto.nullable) = false];

reserved 1, 2, 4, 6, 7, 8, 13, 14, 15, 16, 19;
}

Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/opt/exec/execbuilder/testdata/dist_vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ regions: <hidden>
│ KV bytes read: 40 B
│ KV gRPC calls: 5
│ estimated max memory allocated: 0 B
│ estimated max sql temp disk usage: 0 B
│ missing stats
│ table: kv@kv_pkey
│ spans: FULL SCAN
Expand All @@ -131,11 +132,12 @@ regions: <hidden>
KV bytes read: 40 B
KV gRPC calls: 5
estimated max memory allocated: 0 B
estimated max sql temp disk usage: 0 B
missing stats
table: kw@kw_pkey
spans: FULL SCAN
·
Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzsmt1u2zYUx-_3FASv2lWuTX04toACxtIOSLckRRL0ZggKRjqxBcmiK9J2vCCPtRfYkw2S5yS2LM9HcyVDUi6KSqQ-zu_w_PknrUcqvwfUpteffv90ekN-Jr9eXZ4Tf0Y-X55dEH9OLi-IP3vvkw_En7_3qUZD4cIFH4Ok9h-UUY3qVKMG1ahJNWrRW41OIuGAlCKKuzwmF5y5D9TuaNQLJ1MVn77VqCMioPYjVZ4KgNr0ht8FcAXchajdoRp1QXEvSB7jzwb-7NvEhwXV6KkIpuNQ2sTXyIxq9HrC46NWO36R374S5Y3BJp2__5LLY0eECkLliTDVFIm5JC44wgXXJmx58m6hQJIIuGuTHvlleXJ49eWUODwI5HO_CfeiVb_40edfT0-JVDAhjpiGiryBB9X2QvXWJp0koGUHAD-rw5g_kDGMRbQgPAiEw1X8Wp3kHe64ckYgiZiqyVTZJO6fvP7qBKO3TxpdHi3Br8DeLciIy9E60kHc_1ajUvEhUJu9StvZR2p3nrR8metuZG4-8OfbMjd_yVybNbn7P7nTN3LXzczdy31F5EIE7uZ938UvslevLcPgHKIhfBZeCFGbbVRwAPfqzYC9e_sh8oaj5X-pRi_j6AdMG-jawNAGsYTAAzjTdMZ38Y3b5PeAKBhPiOtJn0wlH0Ju_C9ojQ20LLsuXjhMw23ctiK7EC0xaTNrHVYGgv-Mw8qMw9yMw1qLg-1f3wyvzG291Taa-j6YNrO8uTvJoc1N7g6qzSeZuStQm1k1tXm9LvT960LPoWlGq202dXEwTdPz5q6XQ9Oa3B1U03qZuStQ0_Rqatp6XRj714WRQ9PMVttq6uJgmmbkzV0_h6Y1uTuopvUzc1egphnV1LT1ujD3rwszh6ZZraYqDqZoZt7MWXhFa7UZ4aFLGBFqBFGTxYNpm5WZxQK1zaymtpmYffMrkBMRStjYJ9z-qM7Go1osZgzuEJaZk2IaOfAlEk7Sd3l4mdwoKWQXpFq26suDs3DVJBVXuzYefyTbDk1CjICPn3932D-UbmYoIai5iHwScAWhs3iOZXV-zj21HqULEiKPB96fPI1gdVny7hE44M0SBq-aVtry3JYAWLWOQcZcNi_G8mFIQP2SAbEdgHoHAcTWAelIQCy7Gio6hAwsIatuhMwUIYYQVP2oBLWLC6WbGUpFc82QgPolAypeUJGAWHY1VHQIGVhCVt0ImSlC-iahzmtCRnrGWd3KSmmzsfNWbF3ROkclzqlYzJzGvaLjhuH4ZK8Gyk81Q8ZStnEvPNc6ElANjTuWUNnGvXCvYqYIWTmNe0WH0AmOT_Zq4AgEFRlL2ca9eEFFAqqhcccSKtu4Fy-oKUJdrHGvNiArNeOcNBZ-TYVwfOq3oY8ElL0uKH9K1pGxlG7hC5cLA0uobAtf_FZRilCvsfCvAfVwfOq3oY8ElL0uOAJBRcZSuoUvXlCxhMq28MULaopQv7HwaxY-NeOwvN_PVHQIGUhAZZv4wocQwxKq3-4-ltCOtUH587KBjqZ-Pj6NKO9HNBUl1EcCKtvJF6-qWEL12-LHEtqxQDgCVUVHUz8zn0aE_pKm2qJhpeed1AdCR-Xmi_9NBwmobDdf_A4SllD93DyWUOk7_cWvmdGIspcI5U_NZjoazKeKNTCrFhJQ2W6-eFXFEqqfm8cSKn27v3hVRSPKXiIcgaqmo0l9gFRzN28-3Wr0PhDzb55Lbdr596-15Z_VH40v4ENJ7Ud6PRLzhNbNYgKS2vc8kKDRc-7DR1AQjb3Qk8pzqK2iKTw9_fRPAAAA__88tZtc
Diagram: https://cockroachdb.github.io/distsqlplan/decode.html#eJzsmt1u2zYUx-_3FASv2lWuTX04toACxtIOSLckRVr0ZggKRjqxBcmiK9J2vCCPtRfYkw2S5yS2rMxHWClDUi6KSqQ-zu_w_PknrXsqv0fUpZ8__P7h9Av5mfx6dXlOwgX5eHl2QcIlubwg4eJtSN6RcPk2pAaNhQ8XfAqSun9QRg1qUoNa1KA2NahDrw06S4QHUook7XKfXXDm31G3Z9Agns1VevraoJ5IgLr3VAUqAurSL_wmgivgPiTdHjWoD4oHUfaYcDEKF99mIayoQU9FNJ_G0iWhQRbUoJ9nPD3qdNMX-e0rUcEUXNL7-y-5PvZErCBWgYhzTYlYSuKDJ3zwXcLWJ29WCiRJgPsuGZBf1ifHV59OicejSD72m_Eg2fRLH33-9fSUSAUz4ol5rMgruFPdIFavXdLLAlp3AAiLOkz5HZnCVCQrwqNIeFylr9XL3iFtk98jomA6I34gQzKXfAyb5huuvAlIIuZqNlcuSW-XRbc5wej1g0HXR-u8bLjfrMiEy8k28VHa_9qgUvExUJc9y-rZe-r2Hoxyie3vJHY5Cpf7Ert8SmyXtan9gak1d1LbL0zt031F4kMC_u5936QvclCvPaPkHJIxfBRBDEmX7dR_BLfq1Yi9ef0uCcaT9X-pQS_T6EfMGJnGyDJGqQDBHXjz_IDQif8JrbWDlhWXzROHebyP215kF6IjZl3mbMMqQPCfcTiFcdi7cThbcbDDy5_hdb1rdrpWW_66lJ2VTe1JCWVvU6tT2U8KU6tR2Vk9lX27bMzDy8YsoYhWp2u3ZaNLEc2yqR2UUMQ2tToVcVCYWo2KaNZTEbfLxjq8bKwSimh3uk5bNroU0Sqb2mEJRWxTq1MRh4Wp1aiIVj0Vcbts7MPLxi6hiE6nLRpdemiXTayD18NOlxEe-4QRoSaQtEnWpYxOYZI1KqNdT2W0MT8nXIGciVjCzv7o_kf1dh7VYSlj8MewzpwU88SDT4nwsr7rw8vsRlmd-yDVutVcH5zFmyapuHppw_VHsu3RLMQE-PTx55jDQ-kXhhKDWookJBFXEHurx1g255c8UNtR-iAhCXgU_MnzCDaXZe-egAfBImPwrGkjPY9tGYBN6xRkymX3YiwfhgQ0rBgQewHQ4H8BxLYBmUhArLgaajqELCwhp2mE7BwhhhBU86gEtY8LpV8YSk1zzZCAhhUD0i-oSECsuBpqOoQsLCGnaYTsHCFzl1DvOSErP-NsbuXktNl68VZsW9F6RyXOuVjsksa9puOG4fgUrwaqTzVDxlK1cdeeaxMJqIHGHUuoauOu3avYOUJOSeNe0yF0guNTvBo4AkFFxlK1cdcvqEhADTTuWEJVG3f9gpoj1Mca93oDcnIzzklr4bdUCMeneRv6SEDF64Lqp2QTGUvlFl67XFhYQlVbeP1bRTlCg9bCPwc0wPFp3oY-ElDxuuAIBBUZS-UWXr-gYglVbeH1C2qO0LC18FsWPjfjsLLfz9R0CFlIQFWbeO1DiGEJNW93H0vohbVB9fOyhY6meT4-j6jsRzQ1JTREAqrayetXVSyh5m3xYwm9sEA4AlVFR9M8M59HhP6Spt6i4eTnndwHQkfl5vX_poMEVLWb17-DhCXUPDePJVT5Tr_-NTMaUfESofqp2c5Hg_lUsQFm1UECqtrN61dVLKHmuXksocq3-_WrKhpR8RLhCFQ1H03uA6SGu3n74dqgt5FYfgt86tLev3-dPf9s_mh6AR9L6t7TzxOxzGh9Wc1AUveWRxIMes5DeA8KkmkQB1IFHnVVMoeHh5_-CQAA___IlP5s

query T
EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv
Expand Down
Loading

0 comments on commit 798c6b1

Please sign in to comment.